From b40bb26e40bf70422696dcfccb84a7ce314951d2 Mon Sep 17 00:00:00 2001 From: kim-up Date: Thu, 29 Jun 2023 09:27:37 +0800 Subject: [PATCH 01/30] [Server][Metric] add prometheus --- .../seatunnel-engine-server/pom.xml | 26 +++++++++++++++++++ .../engine/server/SeaTunnelServerStarter.java | 9 ++++++- 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/seatunnel-engine/seatunnel-engine-server/pom.xml b/seatunnel-engine/seatunnel-engine-server/pom.xml index b31756674bf..00dfff2dbff 100644 --- a/seatunnel-engine/seatunnel-engine-server/pom.xml +++ b/seatunnel-engine/seatunnel-engine-server/pom.xml @@ -82,6 +82,32 @@ optional provided + + + + io.prometheus + simpleclient + 0.16.0 + + + + io.prometheus + simpleclient_hotspot + 0.16.0 + + + + io.prometheus + simpleclient_httpserver + 0.16.0 + + + + io.prometheus + simpleclient_pushgateway + 0.16.0 + + diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 4102ace4f25..04305b3073a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -17,6 +17,9 @@ package org.apache.seatunnel.engine.server; +import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.hotspot.DefaultExports; +import java.io.IOException; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; @@ -27,8 +30,12 @@ public class SeaTunnelServerStarter { - public static void main(String[] args) { + public static void main(String[] args) throws IOException { createHazelcastInstance(); + DefaultExports.initialize(); + HTTPServer httpServer = new HTTPServer.Builder() + .withPort(1234) + .build(); } public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { From 1d318281f2170842ca51df3d24401911d068c7bf Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 12 Jul 2023 18:16:38 +0800 Subject: [PATCH 02/30] [Server-core][telemetry] add telemetry core --- config/seatunnel.yaml | 6 +- docs/en/seatunnel-engine/telemetry.md | 45 +++++ .../engine/common/config/EngineConfig.java | 3 + .../YamlSeaTunnelDomConfigProcessor.java | 37 ++++ .../config/server/ServerConfigOptions.java | 24 +++ .../common/config/server/TelemetryConfig.java | 29 +++ .../config/server/TelemetryMetricConfig.java | 38 ++++ .../src/main/resources/seatunnel.yaml | 4 + .../engine/server/CoordinatorService.java | 173 +++++++++--------- .../engine/server/SeaTunnelServerStarter.java | 22 ++- .../telemetry/metrics/AbstractCollector.java | 63 +++++++ .../telemetry/metrics/ExportsInstance.java | 58 ++++++ .../metrics/ExportsInstanceFactory.java | 34 ++++ .../telemetry/metrics/entity/JobCounter.java | 37 ++++ .../metrics/entity/ThreadPoolStatus.java | 32 ++++ .../metrics/exports/JobMetricExports.java | 49 +++++ .../exports/ThreadPoolStatusExports.java | 48 +++++ 17 files changed, 610 insertions(+), 92 deletions(-) create mode 100644 docs/en/seatunnel-engine/telemetry.md create mode 100644 seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java create mode 100644 seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java diff --git a/config/seatunnel.yaml b/config/seatunnel.yaml index 7e496ca39ad..7936714c9df 100644 --- a/config/seatunnel.yaml +++ b/config/seatunnel.yaml @@ -34,4 +34,8 @@ seatunnel: plugin-config: namespace: /tmp/seatunnel/checkpoint_snapshot storage.type: hdfs - fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission \ No newline at end of file + fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission + telemetry: + metric: + http-port: 1025 + load-default-exports: false \ No newline at end of file diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md new file mode 100644 index 00000000000..4e994acacf9 --- /dev/null +++ b/docs/en/seatunnel-engine/telemetry.md @@ -0,0 +1,45 @@ +--- + +sidebar_position: 8 +------------------- + +# Telemetry + +Integrating `Open Telemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. + +You can configure the port exposed by the telemetry server in the `seatunnel.yaml` file. + +The following is an example declarative configuration. + +```yaml +seatunnel: + engine: + telemetry: + http-port: 1024 +``` + +*********************************************** + Job info detail +*********************************************** +createdJobCount : 0 +scheduledJobCount : 0 +runningJobCount : 0 +failingJobCount : 0 +failedJobCount : 0 +cancellingJobCount : 0 +canceledJobCount : 0 +finishedJobCount : 0 +restartingJobCount : 0 +suspendedJobCount : 0 +reconcilingJobCount : 0 + +*********************************************** + CoordinatorService Thread Pool Status +*********************************************** +activeCount : 0 +corePoolSize : 0 +maximumPoolSize : 2147483647 +poolSize : 0 +completedTaskCount : 0 +taskCount : 0 +*********************************************** diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java index edc18a0b15e..1f5731c201f 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.engine.common.config.server.QueueType; import org.apache.seatunnel.engine.common.config.server.ServerConfigOptions; import org.apache.seatunnel.engine.common.config.server.SlotServiceConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryConfig; import org.apache.seatunnel.engine.common.config.server.ThreadShareMode; import lombok.Data; @@ -49,6 +50,8 @@ public class EngineConfig { private CheckpointConfig checkpointConfig = ServerConfigOptions.CHECKPOINT.defaultValue(); + private TelemetryConfig telemetryConfig = ServerConfigOptions.TELEMETRY.defaultValue(); + private QueueType queueType = ServerConfigOptions.QUEUE_TYPE.defaultValue(); public void setBackupCount(int newBackupCount) { diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java index 718e915a0c3..2e602574357 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.engine.common.config.server.QueueType; import org.apache.seatunnel.engine.common.config.server.ServerConfigOptions; import org.apache.seatunnel.engine.common.config.server.SlotServiceConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.common.config.server.ThreadShareMode; import org.w3c.dom.Node; @@ -131,6 +133,8 @@ private void parseEngineConfig(Node engineNode, SeaTunnelConfig config) { engineConfig.setSlotServiceConfig(parseSlotServiceConfig(node)); } else if (ServerConfigOptions.CHECKPOINT.key().equals(name)) { engineConfig.setCheckpointConfig(parseCheckpointConfig(node)); + } else if (ServerConfigOptions.TELEMETRY.key().equals(name)) { + engineConfig.setTelemetryConfig(parseTelemetryConfig(node)); } else { LOGGER.warning("Unrecognized element: " + name); } @@ -206,4 +210,37 @@ private Map parseCheckpointPluginConfig(Node checkpointPluginCon } return checkpointPluginConfig; } + + private TelemetryConfig parseTelemetryConfig(Node telemetryNode) { + TelemetryConfig metricConfig = new TelemetryConfig(); + for (Node node : childElements(telemetryNode)) { + String name = cleanNodeName(node); + if (ServerConfigOptions.TELEMETRY_METRIC.key().equals(name)) { + metricConfig.setMetric(parseTelemetryMetricConfig(node)); + } else { + LOGGER.warning("Unrecognized element: " + name); + } + } + + return metricConfig; + } + + private TelemetryMetricConfig parseTelemetryMetricConfig(Node metricNode) { + TelemetryMetricConfig metricConfig = new TelemetryMetricConfig(); + for (Node node : childElements(metricNode)) { + String name = cleanNodeName(node); + if (ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key().equals(name)) { + metricConfig.setHttpPort( + getIntegerValue( + ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key(), + getTextContent(node))); + } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.key().equals(name)) { + metricConfig.setLoadDefaultExports(getBooleanValue(getTextContent(node))); + } else { + LOGGER.warning("Unrecognized element: " + name); + } + } + + return metricConfig; + } } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index b5d02c03443..2a0b64dbc25 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -138,4 +138,28 @@ public class ServerConfigOptions { .type(new TypeReference>() {}) .noDefaultValue() .withDescription("The checkpoint storage instance configuration."); + + public static final Option TELEMETRY_METRIC_HTTP_PORT = + Options.key("http-port") + .intType() + .defaultValue(1024) + .withDescription("The telemetry metric server's port."); + + public static final Option TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS = + Options.key("load-default-exports") + .booleanType() + .defaultValue(true) + .withDescription("Whether to load default jvm exports."); + + public static final Option TELEMETRY_METRIC = + Options.key("metric") + .type(new TypeReference() {}) + .defaultValue(new TelemetryMetricConfig()) + .withDescription("The telemetry metric configuration."); + + public static final Option TELEMETRY = + Options.key("telemetry") + .type(new TypeReference() {}) + .defaultValue(new TelemetryConfig()) + .withDescription("The telemetry configuration."); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java new file mode 100644 index 00000000000..45236e3660e --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.common.config.server; + +import lombok.Data; + +import java.io.Serializable; + +@Data +public class TelemetryConfig implements Serializable { + + private TelemetryMetricConfig metric = ServerConfigOptions.TELEMETRY_METRIC.defaultValue(); + +} diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java new file mode 100644 index 00000000000..41fa01a5f0d --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.common.config.server; + +import java.io.Serializable; +import lombok.Data; + +import static com.google.common.base.Preconditions.checkArgument; + +@Data +public class TelemetryMetricConfig implements Serializable { + + private int httpPort = ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.defaultValue(); + private boolean loadDefaultExports = + ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.defaultValue(); + + public void setHttpPort(int httpPort) { + checkArgument( + httpPort >= 0, + "The number of http's port failed checkpoints must be a natural number."); + this.httpPort = httpPort; + } +} diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index e5d92281da7..f3dd3c7179a 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -34,3 +34,7 @@ seatunnel: namespace: /tmp/seatunnel/checkpoint_snapshot/ storage.type: hdfs fs.defaultFS: file:///tmp/ + telemetry: + metric: + http-port: 1025 + load-default-exports: false diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index d2931d0c37e..24aa8780f94 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -48,6 +48,8 @@ import org.apache.seatunnel.engine.server.resourcemanager.ResourceManagerFactory; import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile; import org.apache.seatunnel.engine.server.task.operation.GetMetricsOperation; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -696,31 +698,54 @@ public void memberRemoved(MembershipServiceEvent event) { } public void printExecutionInfo() { - ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) executorService; - int activeCount = threadPoolExecutor.getActiveCount(); - int corePoolSize = threadPoolExecutor.getCorePoolSize(); - int maximumPoolSize = threadPoolExecutor.getMaximumPoolSize(); - int poolSize = threadPoolExecutor.getPoolSize(); - long completedTaskCount = threadPoolExecutor.getCompletedTaskCount(); - long taskCount = threadPoolExecutor.getTaskCount(); + ThreadPoolStatus threadPoolStatus = getThreadPoolStatusMetrics(); logger.info( StringFormatUtils.formatTable( "CoordinatorService Thread Pool Status", "activeCount", - activeCount, + threadPoolStatus.getActiveCount(), "corePoolSize", - corePoolSize, + threadPoolStatus.getCorePoolSize(), "maximumPoolSize", - maximumPoolSize, + threadPoolStatus.getMaximumPoolSize(), "poolSize", - poolSize, + threadPoolStatus.getPoolSize(), "completedTaskCount", - completedTaskCount, + threadPoolStatus.getCompletedTaskCount(), "taskCount", - taskCount)); + threadPoolStatus.getTaskCount())); } public void printJobDetailInfo() { + JobCounter jobCounter = getJobCountMetrics(); + logger.info( + StringFormatUtils.formatTable( + "Job info detail", + "createdJobCount", + jobCounter.getCreatedJobCount(), + "scheduledJobCount", + jobCounter.getScheduledJobCount(), + "runningJobCount", + jobCounter.getRunningJobCount(), + "failingJobCount", + jobCounter.getFailingJobCount(), + "failedJobCount", + jobCounter.getFailedJobCount(), + "cancellingJobCount", + jobCounter.getCancellingJobCount(), + "canceledJobCount", + jobCounter.getCanceledJobCount(), + "finishedJobCount", + jobCounter.getFailedJobCount(), + "restartingJobCount", + jobCounter.getRestartingJobCount(), + "suspendedJobCount", + jobCounter.getSuspendedJobCount(), + "reconcilingJobCount", + jobCounter.getReconcilingJobCount())); + } + + public JobCounter getJobCountMetrics() { AtomicLong createdJobCount = new AtomicLong(); AtomicLong scheduledJobCount = new AtomicLong(); AtomicLong runningJobCount = new AtomicLong(); @@ -735,76 +760,60 @@ public void printJobDetailInfo() { if (runningJobInfoIMap != null) { runningJobInfoIMap - .keySet() - .forEach( - jobId -> { - if (runningJobStateIMap.get(jobId) != null) { - JobStatus jobStatus = - (JobStatus) runningJobStateIMap.get(jobId); - switch (jobStatus) { - case CREATED: - createdJobCount.addAndGet(1); - break; - case SCHEDULED: - scheduledJobCount.addAndGet(1); - break; - case RUNNING: - runningJobCount.addAndGet(1); - break; - case FAILING: - failingJobCount.addAndGet(1); - break; - case FAILED: - failedJobCount.addAndGet(1); - break; - case CANCELLING: - cancellingJobCount.addAndGet(1); - break; - case CANCELED: - canceledJobCount.addAndGet(1); - break; - case FINISHED: - finishedJobCount.addAndGet(1); - break; - case RESTARTING: - restartingJobCount.addAndGet(1); - break; - case SUSPENDED: - suspendedJobCount.addAndGet(1); - break; - case RECONCILING: - reconcilingJobCount.addAndGet(1); - break; - default: - } - } - }); + .keySet() + .forEach( + jobId -> { + if (runningJobStateIMap.get(jobId) != null) { + JobStatus jobStatus = + (JobStatus) runningJobStateIMap.get(jobId); + switch (jobStatus) { + case CREATED: + createdJobCount.addAndGet(1); + break; + case SCHEDULED: + scheduledJobCount.addAndGet(1); + break; + case RUNNING: + runningJobCount.addAndGet(1); + break; + case FAILING: + failingJobCount.addAndGet(1); + break; + case FAILED: + failedJobCount.addAndGet(1); + break; + case CANCELLING: + cancellingJobCount.addAndGet(1); + break; + case CANCELED: + canceledJobCount.addAndGet(1); + break; + case FINISHED: + finishedJobCount.addAndGet(1); + break; + case RESTARTING: + restartingJobCount.addAndGet(1); + break; + case SUSPENDED: + suspendedJobCount.addAndGet(1); + break; + case RECONCILING: + reconcilingJobCount.addAndGet(1); + break; + default: + } + } + }); } + return new JobCounter(createdJobCount.longValue(), scheduledJobCount.longValue(), runningJobCount.longValue() + , failingJobCount.longValue(), failedJobCount.longValue(), cancellingJobCount.longValue(), canceledJobCount.longValue() + , finishedJobCount.longValue(), restartingJobCount.longValue(), suspendedJobCount.longValue(), reconcilingJobCount.longValue()); + } - logger.info( - StringFormatUtils.formatTable( - "Job info detail", - "createdJobCount", - createdJobCount, - "scheduledJobCount", - scheduledJobCount, - "runningJobCount", - runningJobCount, - "failingJobCount", - failingJobCount, - "failedJobCount", - failedJobCount, - "cancellingJobCount", - cancellingJobCount, - "canceledJobCount", - canceledJobCount, - "finishedJobCount", - finishedJobCount, - "restartingJobCount", - restartingJobCount, - "suspendedJobCount", - suspendedJobCount, - "reconcilingJobCount", - reconcilingJobCount)); + public ThreadPoolStatus getThreadPoolStatusMetrics(){ + ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) executorService; + return new ThreadPoolStatus(threadPoolExecutor.getActiveCount(), threadPoolExecutor.getCorePoolSize() + , threadPoolExecutor.getMaximumPoolSize(), threadPoolExecutor.getPoolSize() + , threadPoolExecutor.getCompletedTaskCount(), threadPoolExecutor.getTaskCount()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 04305b3073a..7e37a590d31 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -17,25 +17,22 @@ package org.apache.seatunnel.engine.server; -import io.prometheus.client.exporter.HTTPServer; -import io.prometheus.client.hotspot.DefaultExports; -import java.io.IOException; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstanceFactory; import com.hazelcast.instance.impl.HazelcastInstanceFactory; import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.instance.impl.HazelcastInstanceProxy; +import com.hazelcast.instance.impl.Node; import lombok.NonNull; +import java.io.IOException; + public class SeaTunnelServerStarter { public static void main(String[] args) throws IOException { createHazelcastInstance(); - DefaultExports.initialize(); - HTTPServer httpServer = new HTTPServer.Builder() - .withPort(1234) - .build(); } public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { @@ -55,8 +52,15 @@ public static HazelcastInstanceImpl createHazelcastInstance( .getOriginal(); } - public static HazelcastInstanceImpl createHazelcastInstance() { + public static HazelcastInstanceImpl createHazelcastInstance() throws IOException { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); - return createHazelcastInstance(seaTunnelConfig); + HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); + createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + return hazelcastInstance; + } + + private static void createTelemetryInstance( + @NonNull Node node, final SeaTunnelConfig seaTunnelConfig) throws IOException { + ExportsInstanceFactory.newExportsInstance(node, seaTunnelConfig); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java new file mode 100644 index 00000000000..3df2139731c --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics; + +import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.instance.impl.Node; +import com.hazelcast.logging.ILogger; +import io.prometheus.client.Collector; +import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.SeaTunnelServer; + +public abstract class AbstractCollector extends Collector { + + protected ExportsInstance exportsInstance; + + public AbstractCollector(final ExportsInstance exportsInstance) { + this.exportsInstance = exportsInstance; + } + + protected Node getNode() { + return exportsInstance.getNode(); + } + + protected ILogger getLogger(Class clazz) { + return getNode().getLogger(clazz); + } + + protected boolean isMaster() { + return getNode().isMaster(); + } + + protected MemberImpl getLocalMember() { + return getNode().nodeEngine.getLocalMember(); + } + + protected SeaTunnelServer getServer() { + return getNode().getNodeEngine().getService(SeaTunnelServer.SERVICE_NAME); + } + + protected CoordinatorService getCoordinatorService() { + if (isMaster()) { + return getServer().getCoordinatorService(); + } else { + throw new SeaTunnelEngineException("This is not a master node now."); + } + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java new file mode 100644 index 00000000000..ff8c38e0030 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics; + +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; + +import com.hazelcast.instance.impl.Node; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.hotspot.DefaultExports; + +import java.io.IOException; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; + +public class ExportsInstance { + + private Node node; + private TelemetryMetricConfig metricConfig; + + public ExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) throws IOException { + this.node = node; + this.metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); + start(); + } + + private void start() throws IOException { + if(metricConfig.isLoadDefaultExports()){ + DefaultExports.initialize(); + } + HTTPServer httpServer = + new HTTPServer.Builder().withPort(metricConfig.getHttpPort()).build(); + CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; + // + new JobMetricExports(this).register(collectorRegistry); + new ThreadPoolStatusExports(this).register(collectorRegistry); + } + + public Node getNode() { + return node; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java new file mode 100644 index 00000000000..29a85469cec --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.telemetry.metrics; + +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; + +import com.hazelcast.instance.impl.Node; + +import java.io.IOException; + +public final class ExportsInstanceFactory { + + private ExportsInstanceFactory() {} + + public static ExportsInstance newExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) + throws IOException { + return new ExportsInstance(node, seaTunnelConfig); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java new file mode 100644 index 00000000000..09230e27467 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics.entity; + +import lombok.AllArgsConstructor; +import lombok.Data; + +@Data +@AllArgsConstructor +public class JobCounter { + private long createdJobCount; + private long scheduledJobCount; + private long runningJobCount; + private long failingJobCount; + private long failedJobCount; + private long cancellingJobCount; + private long canceledJobCount; + private long finishedJobCount; + private long restartingJobCount; + private long suspendedJobCount; + private long reconcilingJobCount; +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.java new file mode 100644 index 00000000000..ecbf341145d --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.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.seatunnel.engine.server.telemetry.metrics.entity; + +import lombok.AllArgsConstructor; +import lombok.Data; + +@Data +@AllArgsConstructor +public class ThreadPoolStatus { + private int activeCount; + private int corePoolSize; + private int maximumPoolSize; + private int poolSize; + private long completedTaskCount; + private long taskCount; +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java new file mode 100644 index 00000000000..e27d69141e8 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics.exports; + +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; + +import io.prometheus.client.GaugeMetricFamily; + +import java.util.ArrayList; +import java.util.List; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; + +public class JobMetricExports extends AbstractCollector { + + public JobMetricExports(ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + // Only the master can get job metrics + if (isMaster()) { + CoordinatorService coordinatorService = getCoordinatorService(); + JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); + // TODO + mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); + } + return mfs; + } +} + diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java new file mode 100644 index 00000000000..4e070f3b22b --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.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.seatunnel.engine.server.telemetry.metrics.exports; + +import io.prometheus.client.GaugeMetricFamily; +import java.util.ArrayList; +import java.util.List; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; + +public class ThreadPoolStatusExports extends AbstractCollector { + + public ThreadPoolStatusExports(ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + // Only the master can get job metrics + if (isMaster()) { + CoordinatorService coordinatorService = getCoordinatorService(); + ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); + // TODO + mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); + } + return mfs; + } +} + From 61667993ee61df1f8b574d02bdde6d98d2397bff Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 12 Jul 2023 22:25:28 +0800 Subject: [PATCH 03/30] [Server-core][telemetry] add prometheus license and notice --- docs/en/seatunnel-engine/telemetry.md | 8 +- pom.xml | 22 ++++ seatunnel-dist/release-docs/LICENSE | 7 ++ seatunnel-dist/release-docs/NOTICE | 16 +++ .../YamlSeaTunnelDomConfigProcessor.java | 4 +- .../config/server/ServerConfigOptions.java | 16 +-- .../common/config/server/TelemetryConfig.java | 1 - .../config/server/TelemetryMetricConfig.java | 3 +- .../src/main/resources/seatunnel.yaml | 2 +- .../seatunnel-engine-server/pom.xml | 11 +- .../engine/server/CoordinatorService.java | 115 ++++++++++-------- .../telemetry/metrics/AbstractCollector.java | 12 +- .../telemetry/metrics/ExportsInstance.java | 6 +- .../metrics/exports/JobMetricExports.java | 39 +++++- .../exports/ThreadPoolStatusExports.java | 72 +++++++++-- tools/dependencies/known-dependencies.txt | 7 ++ 16 files changed, 239 insertions(+), 102 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 4e994acacf9..9ad28f45c78 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -19,7 +19,8 @@ seatunnel: ``` *********************************************** - Job info detail + +Job info detail *********************************************** createdJobCount : 0 scheduledJobCount : 0 @@ -34,8 +35,11 @@ suspendedJobCount : 0 reconcilingJobCount : 0 *********************************************** - CoordinatorService Thread Pool Status + + CoordinatorService Thread Pool Status + *********************************************** + activeCount : 0 corePoolSize : 0 maximumPoolSize : 2147483647 diff --git a/pom.xml b/pom.xml index 51b03a26d5b..d3532de64ba 100644 --- a/pom.xml +++ b/pom.xml @@ -146,6 +146,9 @@ 2.4.7 3.1.4 4.1.60.Final + + + 0.16.0 @@ -485,6 +488,25 @@ ${netty-buffer.version} + + + io.prometheus + simpleclient + ${prometheus.simpleclient.version} + + + + io.prometheus + simpleclient_hotspot + ${prometheus.simpleclient.version} + + + + io.prometheus + simpleclient_httpserver + ${prometheus.simpleclient.version} + + diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index 5d19a35a874..dca007e2f27 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -275,6 +275,13 @@ The text of each license is the standard Apache 2.0 license. (Apache-2.0) listenablefuture (com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava https://mvnrepository.com/artifact/com.google.guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava) (Apache-2.0) accessors-smart (com.google.guava:accessors-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/accessors-smart) (Apache-2.0) json-smart (net.minidev:json-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/json-smart) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient (io.prometheus:simpleclient:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Common (io.prometheus:simpleclient_common:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Hotspot (io.prometheus:simpleclient_hotspot:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Httpserver (io.prometheus:simpleclient_httpserver:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - Common (io.prometheus:simpleclient_tracer_common:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry (io.prometheus:simpleclient_tracer_otel:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry Agent (io.prometheus:simpleclient_tracer_otel_agent:0.16.0 - https://www.apache.org/hive-storage-api/) ======================================================================== MOZILLA PUBLIC LICENSE License diff --git a/seatunnel-dist/release-docs/NOTICE b/seatunnel-dist/release-docs/NOTICE index 4bed2d4be19..cbbf8cc6549 100644 --- a/seatunnel-dist/release-docs/NOTICE +++ b/seatunnel-dist/release-docs/NOTICE @@ -706,4 +706,20 @@ and updated within the WildFly project (https://github.com/wildfly/wildfly). The class org.apache.calcite.linq4j.tree.ConstantExpression contains code originating from the Calcite project (https://github.com/apache/calcite). +========================================================================= + +Prometheus NOTICE + +========================================================================= +Prometheus instrumentation library for JVM applications +Copyright 2012-2015 The Prometheus Authors + +This product includes software developed at +Boxever Ltd. (http://www.boxever.com/). + +This product includes software developed at +SoundCloud Ltd. (http://soundcloud.com/). + +This product includes software developed as part of the +Ocelli project by Netflix Inc. (https://github.com/Netflix/ocelli/). ========================================================================= \ No newline at end of file diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java index 2e602574357..f22070d69ff 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java @@ -234,7 +234,9 @@ private TelemetryMetricConfig parseTelemetryMetricConfig(Node metricNode) { getIntegerValue( ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key(), getTextContent(node))); - } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.key().equals(name)) { + } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS + .key() + .equals(name)) { metricConfig.setLoadDefaultExports(getBooleanValue(getTextContent(node))); } else { LOGGER.warning("Unrecognized element: " + name); diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index 2a0b64dbc25..ccb12e66f7a 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -152,14 +152,14 @@ public class ServerConfigOptions { .withDescription("Whether to load default jvm exports."); public static final Option TELEMETRY_METRIC = - Options.key("metric") - .type(new TypeReference() {}) - .defaultValue(new TelemetryMetricConfig()) - .withDescription("The telemetry metric configuration."); + Options.key("metric") + .type(new TypeReference() {}) + .defaultValue(new TelemetryMetricConfig()) + .withDescription("The telemetry metric configuration."); public static final Option TELEMETRY = - Options.key("telemetry") - .type(new TypeReference() {}) - .defaultValue(new TelemetryConfig()) - .withDescription("The telemetry configuration."); + Options.key("telemetry") + .type(new TypeReference() {}) + .defaultValue(new TelemetryConfig()) + .withDescription("The telemetry configuration."); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java index 45236e3660e..c3e603eea4c 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java @@ -25,5 +25,4 @@ public class TelemetryConfig implements Serializable { private TelemetryMetricConfig metric = ServerConfigOptions.TELEMETRY_METRIC.defaultValue(); - } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java index 41fa01a5f0d..c4d10cdd0b5 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java @@ -17,9 +17,10 @@ package org.apache.seatunnel.engine.common.config.server; -import java.io.Serializable; import lombok.Data; +import java.io.Serializable; + import static com.google.common.base.Preconditions.checkArgument; @Data diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index f3dd3c7179a..da4b9857ef4 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -37,4 +37,4 @@ seatunnel: telemetry: metric: http-port: 1025 - load-default-exports: false + load-default-exports: true diff --git a/seatunnel-engine/seatunnel-engine-server/pom.xml b/seatunnel-engine/seatunnel-engine-server/pom.xml index 00dfff2dbff..608bc173880 100644 --- a/seatunnel-engine/seatunnel-engine-server/pom.xml +++ b/seatunnel-engine/seatunnel-engine-server/pom.xml @@ -83,29 +83,20 @@ provided - + io.prometheus simpleclient - 0.16.0 io.prometheus simpleclient_hotspot - 0.16.0 io.prometheus simpleclient_httpserver - 0.16.0 - - - - io.prometheus - simpleclient_pushgateway - 0.16.0 diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index 24aa8780f94..228225bba7c 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -760,60 +760,73 @@ public JobCounter getJobCountMetrics() { if (runningJobInfoIMap != null) { runningJobInfoIMap - .keySet() - .forEach( - jobId -> { - if (runningJobStateIMap.get(jobId) != null) { - JobStatus jobStatus = - (JobStatus) runningJobStateIMap.get(jobId); - switch (jobStatus) { - case CREATED: - createdJobCount.addAndGet(1); - break; - case SCHEDULED: - scheduledJobCount.addAndGet(1); - break; - case RUNNING: - runningJobCount.addAndGet(1); - break; - case FAILING: - failingJobCount.addAndGet(1); - break; - case FAILED: - failedJobCount.addAndGet(1); - break; - case CANCELLING: - cancellingJobCount.addAndGet(1); - break; - case CANCELED: - canceledJobCount.addAndGet(1); - break; - case FINISHED: - finishedJobCount.addAndGet(1); - break; - case RESTARTING: - restartingJobCount.addAndGet(1); - break; - case SUSPENDED: - suspendedJobCount.addAndGet(1); - break; - case RECONCILING: - reconcilingJobCount.addAndGet(1); - break; - default: - } - } - }); + .keySet() + .forEach( + jobId -> { + if (runningJobStateIMap.get(jobId) != null) { + JobStatus jobStatus = + (JobStatus) runningJobStateIMap.get(jobId); + switch (jobStatus) { + case CREATED: + createdJobCount.addAndGet(1); + break; + case SCHEDULED: + scheduledJobCount.addAndGet(1); + break; + case RUNNING: + runningJobCount.addAndGet(1); + break; + case FAILING: + failingJobCount.addAndGet(1); + break; + case FAILED: + failedJobCount.addAndGet(1); + break; + case CANCELLING: + cancellingJobCount.addAndGet(1); + break; + case CANCELED: + canceledJobCount.addAndGet(1); + break; + case FINISHED: + finishedJobCount.addAndGet(1); + break; + case RESTARTING: + restartingJobCount.addAndGet(1); + break; + case SUSPENDED: + suspendedJobCount.addAndGet(1); + break; + case RECONCILING: + reconcilingJobCount.addAndGet(1); + break; + default: + } + } + }); } - return new JobCounter(createdJobCount.longValue(), scheduledJobCount.longValue(), runningJobCount.longValue() - , failingJobCount.longValue(), failedJobCount.longValue(), cancellingJobCount.longValue(), canceledJobCount.longValue() - , finishedJobCount.longValue(), restartingJobCount.longValue(), suspendedJobCount.longValue(), reconcilingJobCount.longValue()); + return new JobCounter( + createdJobCount.longValue(), + scheduledJobCount.longValue(), + runningJobCount.longValue(), + failingJobCount.longValue(), + failedJobCount.longValue(), + cancellingJobCount.longValue(), + canceledJobCount.longValue(), + finishedJobCount.longValue(), + restartingJobCount.longValue(), + suspendedJobCount.longValue(), + reconcilingJobCount.longValue()); } - public ThreadPoolStatus getThreadPoolStatusMetrics(){ + public ThreadPoolStatus getThreadPoolStatusMetrics() { ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) executorService; - return new ThreadPoolStatus(threadPoolExecutor.getActiveCount(), threadPoolExecutor.getCorePoolSize() - , threadPoolExecutor.getMaximumPoolSize(), threadPoolExecutor.getPoolSize() - , threadPoolExecutor.getCompletedTaskCount(), threadPoolExecutor.getTaskCount()); + return new ThreadPoolStatus( + threadPoolExecutor.getActiveCount(), + threadPoolExecutor.getCorePoolSize(), + threadPoolExecutor.getMaximumPoolSize(), + threadPoolExecutor.getPoolSize(), + threadPoolExecutor.getCompletedTaskCount(), + threadPoolExecutor.getTaskCount()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java index 3df2139731c..6547a2708b4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -17,13 +17,13 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.SeaTunnelServer; + import com.hazelcast.cluster.impl.MemberImpl; import com.hazelcast.instance.impl.Node; import com.hazelcast.logging.ILogger; import io.prometheus.client.Collector; -import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; -import org.apache.seatunnel.engine.server.CoordinatorService; -import org.apache.seatunnel.engine.server.SeaTunnelServer; public abstract class AbstractCollector extends Collector { @@ -54,10 +54,6 @@ protected SeaTunnelServer getServer() { } protected CoordinatorService getCoordinatorService() { - if (isMaster()) { - return getServer().getCoordinatorService(); - } else { - throw new SeaTunnelEngineException("This is not a master node now."); - } + return getServer().getCoordinatorService(); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java index ff8c38e0030..cfff6d012db 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -19,6 +19,8 @@ import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; import com.hazelcast.instance.impl.Node; import io.prometheus.client.CollectorRegistry; @@ -26,8 +28,6 @@ import io.prometheus.client.hotspot.DefaultExports; import java.io.IOException; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; public class ExportsInstance { @@ -41,7 +41,7 @@ public ExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) throws IOExce } private void start() throws IOException { - if(metricConfig.isLoadDefaultExports()){ + if (metricConfig.isLoadDefaultExports()) { DefaultExports.initialize(); } HTTPServer httpServer = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index e27d69141e8..625380ea107 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -20,12 +20,13 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; public class JobMetricExports extends AbstractCollector { @@ -40,10 +41,40 @@ public List collect() { if (isMaster()) { CoordinatorService coordinatorService = getCoordinatorService(); JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); - // TODO - mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "st_job_count", + "The job count of seatunnel cluster ", + Collections.singletonList("type")); + + metricFamily.addMetric( + Collections.singletonList("canceled"), jobCountMetrics.getCanceledJobCount()); + metricFamily.addMetric( + Collections.singletonList("cancelling"), + jobCountMetrics.getCancellingJobCount()); + metricFamily.addMetric( + Collections.singletonList("created"), jobCountMetrics.getCreatedJobCount()); + metricFamily.addMetric( + Collections.singletonList("failed"), jobCountMetrics.getFailedJobCount()); + metricFamily.addMetric( + Collections.singletonList("failing"), jobCountMetrics.getFailingJobCount()); + metricFamily.addMetric( + Collections.singletonList("finished"), jobCountMetrics.getFinishedJobCount()); + metricFamily.addMetric( + Collections.singletonList("reconciling"), + jobCountMetrics.getReconcilingJobCount()); + metricFamily.addMetric( + Collections.singletonList("restarting"), + jobCountMetrics.getRestartingJobCount()); + metricFamily.addMetric( + Collections.singletonList("running"), jobCountMetrics.getRunningJobCount()); + metricFamily.addMetric( + Collections.singletonList("scheduled"), jobCountMetrics.getScheduledJobCount()); + metricFamily.addMetric( + Collections.singletonList("suspended"), jobCountMetrics.getSuspendedJobCount()); + + mfs.add(metricFamily); } return mfs; } } - diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java index 4e070f3b22b..1f3c9944e80 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java @@ -17,17 +17,22 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; -import io.prometheus.client.GaugeMetricFamily; -import java.util.ArrayList; -import java.util.List; import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; -import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; +import io.prometheus.client.CounterMetricFamily; +import io.prometheus.client.GaugeMetricFamily; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + public class ThreadPoolStatusExports extends AbstractCollector { + private static String HELP = "The %s of seatunnel coordinator executors cachedThreadPool"; + public ThreadPoolStatusExports(ExportsInstance exportsInstance) { super(exportsInstance); } @@ -35,14 +40,57 @@ public ThreadPoolStatusExports(ExportsInstance exportsInstance) { @Override public List collect() { List mfs = new ArrayList(); - // Only the master can get job metrics - if (isMaster()) { - CoordinatorService coordinatorService = getCoordinatorService(); - ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); - // TODO - mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); - } + CoordinatorService coordinatorService = getCoordinatorService(); + + String address = getNode().getNodeEngine().getLocalMember().getAddress().toString(); + List labelValues = Collections.singletonList(address); + + ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); + List labelNames = Collections.singletonList("address"); + GaugeMetricFamily activeCount = + new GaugeMetricFamily( + "st_thread_pool_activeCount", + String.format(HELP, "activeCount"), + labelNames); + activeCount.addMetric(labelValues, threadPoolStatusMetrics.getActiveCount()); + mfs.add(activeCount); + + CounterMetricFamily completedTask = + new CounterMetricFamily( + "st_thread_pool_completedTask", + String.format(HELP, "completedTask"), + labelNames); + completedTask.addMetric(labelValues, threadPoolStatusMetrics.getCompletedTaskCount()); + mfs.add(completedTask); + + GaugeMetricFamily corePoolSize = + new GaugeMetricFamily( + "st_thread_pool_corePoolSize", + String.format(HELP, "activeCount"), + labelNames); + corePoolSize.addMetric(labelValues, threadPoolStatusMetrics.getCorePoolSize()); + mfs.add(corePoolSize); + + GaugeMetricFamily maximumPoolSize = + new GaugeMetricFamily( + "st_thread_pool_maximumPoolSize", + String.format(HELP, "maximumPoolSize"), + labelNames); + maximumPoolSize.addMetric(labelValues, threadPoolStatusMetrics.getMaximumPoolSize()); + mfs.add(maximumPoolSize); + + GaugeMetricFamily poolSize = + new GaugeMetricFamily( + "st_thread_pool_poolSize", String.format(HELP, "poolSize"), labelNames); + poolSize.addMetric(labelValues, threadPoolStatusMetrics.getPoolSize()); + mfs.add(poolSize); + + CounterMetricFamily taskCount = + new CounterMetricFamily( + "st_thread_pool_task", String.format(HELP, "taskCount"), labelNames); + taskCount.addMetric(labelValues, threadPoolStatusMetrics.getTaskCount()); + mfs.add(taskCount); + return mfs; } } - diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 3a1e736b68b..9833c71fdda 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -38,3 +38,10 @@ listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar accessors-smart-2.4.7.jar asm-9.1.jar json-smart-2.4.7.jar +simpleclient-0.16.0.jar +simpleclient_common-0.16.0.jar +simpleclient_hotspot-0.16.0.jar +simpleclient_httpserver-0.16.0.jar +simpleclient_tracer_common-0.16.0.jar +simpleclient_tracer_otel-0.16.0.jar +simpleclient_tracer_otel_agent-0.16.0.jar From 32f60626af1a20bb55dc73fe1bf0981f3df471b8 Mon Sep 17 00:00:00 2001 From: kim-up Date: Thu, 13 Jul 2023 12:11:39 +0800 Subject: [PATCH 04/30] [Server-core][telemetry] add e2e test --- config/seatunnel.yaml | 2 +- docs/en/seatunnel-engine/telemetry.md | 33 +--- .../seatunnel/engine/e2e/TelemetryApiIT.java | 141 ++++++++++++++++++ .../config/server/ServerConfigOptions.java | 2 +- .../src/main/resources/seatunnel.yaml | 2 +- .../engine/server/SeaTunnelServerStarter.java | 2 +- 6 files changed, 147 insertions(+), 35 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java diff --git a/config/seatunnel.yaml b/config/seatunnel.yaml index 7936714c9df..67c42868e31 100644 --- a/config/seatunnel.yaml +++ b/config/seatunnel.yaml @@ -37,5 +37,5 @@ seatunnel: fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission telemetry: metric: - http-port: 1025 + http-port: 9090 load-default-exports: false \ No newline at end of file diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 9ad28f45c78..a1b63c187ac 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -5,7 +5,7 @@ sidebar_position: 8 # Telemetry -Integrating `Open Telemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. +Integrating `OpenTelemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. You can configure the port exposed by the telemetry server in the `seatunnel.yaml` file. @@ -15,35 +15,6 @@ The following is an example declarative configuration. seatunnel: engine: telemetry: - http-port: 1024 + http-port: 9090 ``` -*********************************************** - -Job info detail -*********************************************** -createdJobCount : 0 -scheduledJobCount : 0 -runningJobCount : 0 -failingJobCount : 0 -failedJobCount : 0 -cancellingJobCount : 0 -canceledJobCount : 0 -finishedJobCount : 0 -restartingJobCount : 0 -suspendedJobCount : 0 -reconcilingJobCount : 0 - -*********************************************** - - CoordinatorService Thread Pool Status - -*********************************************** - -activeCount : 0 -corePoolSize : 0 -maximumPoolSize : 2147483647 -poolSize : 0 -completedTaskCount : 0 -taskCount : 0 -*********************************************** diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java new file mode 100644 index 00000000000..c68da5faeac --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.e2e; + +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.seatunnel.common.config.Common; +import org.apache.seatunnel.common.config.DeployMode; +import org.apache.seatunnel.engine.client.SeaTunnelClient; +import org.apache.seatunnel.engine.client.job.ClientJobProxy; +import org.apache.seatunnel.engine.client.job.JobExecutionEnvironment; +import org.apache.seatunnel.engine.common.config.ConfigProvider; +import org.apache.seatunnel.engine.common.config.JobConfig; +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; +import org.apache.seatunnel.engine.core.job.JobStatus; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; +import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.awaitility.Awaitility; +import org.hamcrest.Matcher; +import org.hamcrest.core.StringContains; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static io.restassured.RestAssured.given; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.matchesRegex; + +@Slf4j +public class TelemetryApiIT { + + private static final String HOST = "http://localhost:"; + + private static ClientJobProxy clientJobProxy; + + private static HazelcastInstanceImpl hazelcastInstance; + + private static TelemetryMetricConfig metricConfig; + + @BeforeAll + static void beforeClass() throws Exception { + String testClusterName = TestUtils.getClusterName("TelemetryApiIT"); + SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); + seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName); + // get TelemetryMetricConfig + metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); + hazelcastInstance = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); + // createTelemetryInstance + SeaTunnelServerStarter.createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + Common.setDeployMode(DeployMode.CLIENT); + String filePath = TestUtils.getResource("stream_fakesource_to_file.conf"); + JobConfig jobConfig = new JobConfig(); + jobConfig.setName("fake_to_file"); + + ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); + clientConfig.setClusterName(testClusterName); + SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig); + JobExecutionEnvironment jobExecutionEnv = + engineClient.createExecutionContext(filePath, jobConfig); + + clientJobProxy = jobExecutionEnv.execute(); + + Awaitility.await() + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + Assertions.assertEquals( + JobStatus.RUNNING, clientJobProxy.getJobStatus())); + } + + @Test + public void testGetMetrics() { + given().get( + HOST + + metricConfig.getHttpPort() + + "/metrics") + .then() + .statusCode(200) + // Use regular expressions to verify whether the response body is the indicator data of Prometheus + // Metric data is usually multi-line, use newlines for validation + .body(matchesRegex("(?s)^.*# HELP.*# TYPE.*$")) + // Verify that the response body contains a specific metric + // JVM metrics + .body(containsString("jvm_threads")) + .body(containsString("jvm_memory_pool")) + .body(containsString("jvm_gc")) + .body(containsString("jvm_info")) + .body(containsString("jvm_memory_bytes")) + .body(containsString("jvm_classes")) + .body(containsString("jvm_buffer_pool")) + .body(containsString("process_start")) + // Thread pool metrics + .body(containsString("st_thread_pool_activeCount{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) + .body(containsString("st_thread_pool_poolSize{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_task_total{address=\"[localhost]:5801\",}")) + // Job count metrics + .body(containsString("st_job_count{type=\"canceled\",} 0.0")) + .body(containsString("st_job_count{type=\"cancelling\",} 0.0")) + .body(containsString("st_job_count{type=\"created\",} 0.0")) + .body(containsString("st_job_count{type=\"failed\",} 0.0")) + .body(containsString("st_job_count{type=\"failing\",} 0.0")) + .body(containsString("st_job_count{type=\"finished\",} 0.0")) + .body(containsString("st_job_count{type=\"reconciling\",} 0.0")) + .body(containsString("st_job_count{type=\"restarting\",} 0.0")) + // Running job count is 1 + .body(containsString("st_job_count{type=\"running\",} 1.0")) + .body(containsString("st_job_count{type=\"scheduled\",} 0.0")) + .body(containsString("st_job_count{type=\"suspended\",} 0.0")); + } + + @AfterAll + static void afterClass() { + if (hazelcastInstance != null) { + hazelcastInstance.shutdown(); + } + } +} diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index ccb12e66f7a..b956d4006b1 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -142,7 +142,7 @@ public class ServerConfigOptions { public static final Option TELEMETRY_METRIC_HTTP_PORT = Options.key("http-port") .intType() - .defaultValue(1024) + .defaultValue(9090) .withDescription("The telemetry metric server's port."); public static final Option TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS = diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index da4b9857ef4..9ceb2ed68a3 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -36,5 +36,5 @@ seatunnel: fs.defaultFS: file:///tmp/ telemetry: metric: - http-port: 1025 + http-port: 9090 load-default-exports: true diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 7e37a590d31..91da8fc362f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -59,7 +59,7 @@ public static HazelcastInstanceImpl createHazelcastInstance() throws IOException return hazelcastInstance; } - private static void createTelemetryInstance( + public static void createTelemetryInstance( @NonNull Node node, final SeaTunnelConfig seaTunnelConfig) throws IOException { ExportsInstanceFactory.newExportsInstance(node, seaTunnelConfig); } From 1f41825ed3ed1d42f1edbf09366764a65ff1ad81 Mon Sep 17 00:00:00 2001 From: kim-up Date: Fri, 14 Jul 2023 23:27:19 +0800 Subject: [PATCH 05/30] [Server-core][telemetry] update telemetry docs --- docs/en/seatunnel-engine/telemetry.md | 69 ++++++++++- release-note.md | 1 + .../seatunnel/core/starter/SeaTunnel.java | 5 +- .../core/starter/command/Command.java | 4 +- .../core/starter/flink/SeaTunnelFlink.java | 4 +- .../core/starter/flink/SeaTunnelFlink.java | 4 +- .../core/starter/spark/SeaTunnelSpark.java | 4 +- .../core/starter/spark/SeaTunnelSpark.java | 4 +- .../starter/seatunnel/SeaTunnelClient.java | 4 +- .../starter/seatunnel/SeaTunnelServer.java | 4 +- .../command/ServerExecuteCommand.java | 11 +- .../seatunnel/engine/e2e/TelemetryApiIT.java | 110 +++++++++--------- .../src/test/resources/hazelcast.yaml | 2 + .../engine/server/SeaTunnelServerStarter.java | 7 +- .../telemetry/metrics/ExportsInstance.java | 7 +- .../metrics/exports/JobMetricExports.java | 2 +- ...s.java => JobThreadPoolStatusExports.java} | 19 +-- .../engine/server/CoordinatorServiceTest.java | 8 +- .../engine/SeaTunnelEngineExample.java | 8 +- .../engine/SeaTunnelEngineServerExample.java | 4 +- .../example/flink/v2/SeaTunnelApiExample.java | 3 +- .../example/spark/v2/ExampleUtils.java | 3 +- .../example/spark/v2/SeaTunnelApiExample.java | 4 +- 23 files changed, 192 insertions(+), 99 deletions(-) rename seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/{ThreadPoolStatusExports.java => JobThreadPoolStatusExports.java} (84%) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index a1b63c187ac..c880c26f8bb 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -7,7 +7,7 @@ sidebar_position: 8 Integrating `OpenTelemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. -You can configure the port exposed by the telemetry server in the `seatunnel.yaml` file. +You can configure telemetry's configurations in the `seatunnel.yaml` file. The following is an example declarative configuration. @@ -15,6 +15,71 @@ The following is an example declarative configuration. seatunnel: engine: telemetry: - http-port: 9090 + http-port: 9090 # The port exposed by the telemetry server, default is 9090. + load-default-exports: true # Whether to load default jvm exports, default is true. ``` +## Metrics + +Available metrics include the following categories. + +### Thread Pool Status + +| MetricName | Type | Labels | +|-------------------------------------|---------|----------------------------------------------------------------------| +| job_thread_pool_activeCount | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_corePoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_maximumPoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_poolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_completedTask_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_task_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | + +### Job info detail + +| MetricName | Type | Labels | +|------------|-------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| job_count | Gauge | **type**, the type of job, including: "canceled" "cancelling" "created" "failed" "failing" "finished" "reconciling" "restarting" "running" "scheduled" "suspended" | + +### JVM Metrics + +| MetricName | Type | Labels | +|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------| +| jvm_threads_current | Gauge | - | +| jvm_threads_daemon | Gauge | - | +| jvm_threads_daemon | Gauge | - | +| jvm_threads_peak | Gauge | - | +| jvm_threads_started_total | Counter | - | +| jvm_threads_deadlocked | Gauge | - | +| jvm_threads_deadlocked_monitor | Gauge | - | +| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | +| jvm_classes_currently_loaded | Gauge | - | +| jvm_classes_loaded_total | Counter | - | +| jvm_classes_unloaded_total | Counter | - | +| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | +| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | +| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | +| process_cpu_seconds_total | Counter | - | +| process_start_time_seconds | Gauge | - | +| process_open_fds | Gauge | - | +| process_max_fds | Gauge | - | +| jvm_memory_objects_pending_finalization | Gauge | - | +| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | +| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | +| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | +| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | +| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | +| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | +| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | + +### 实现方案 + diff --git a/release-note.md b/release-note.md index 0099f07bcf8..8855f544dde 100644 --- a/release-note.md +++ b/release-note.md @@ -161,6 +161,7 @@ - [Zeta] Reduce the frequency of fetching data from imap (#4851) - [Zeta] Add OSS support for Imap storage to cluster-mode type (#4683) - [Zeta] Improve local mode startup request ports (#4660) +- [Zeta] Support exposing jobCount metrics, jobThreadPool metrics and JVM metrics by prometheus exporter (#5070) ## Docs diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java index e003bf2d206..22afb8fc9dc 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java @@ -26,6 +26,8 @@ import lombok.extern.slf4j.Slf4j; +import java.io.IOException; + @Slf4j public class SeaTunnel { @@ -35,7 +37,8 @@ public class SeaTunnel { * @param command commandArgs * @param commandType */ - public static void run(Command command) throws CommandException { + public static void run(Command command) + throws CommandException, IOException { try { command.execute(); } catch (ConfigRuntimeException e) { diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java index 1d510ab984b..e419c89b46b 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.core.starter.exception.CommandExecuteException; import org.apache.seatunnel.core.starter.exception.ConfigCheckException; +import java.io.IOException; + /** * Command interface, only has one method {@link Command#execute()}, used to execute the command * @@ -28,5 +30,5 @@ @FunctionalInterface public interface Command { - void execute() throws CommandExecuteException, ConfigCheckException; + void execute() throws CommandExecuteException, ConfigCheckException, IOException; } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java index 8d1b434801d..460b4f2874d 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelFlink { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java index 1595da686a7..e7baa0e7811 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelFlink { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java index ca7b2ed4be1..2de9c4ba930 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java @@ -23,9 +23,11 @@ import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelSpark { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { SparkCommandArgs sparkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java index 9b3fde6fd10..85cf2005d51 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java @@ -23,9 +23,11 @@ import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelSpark { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { SparkCommandArgs sparkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java index 7a37fd340c3..3a755f808ab 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ClientCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelClient { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { ClientCommandArgs clientCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java index 96a3e32e51a..ae3284fc010 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ServerCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelServer { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { ServerCommandArgs serverCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java index aead1f8a964..e5d11fc3b5c 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java @@ -21,11 +21,11 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ServerCommandArgs; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.server.SeaTunnelNodeContext; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; import org.apache.commons.lang3.StringUtils; -import com.hazelcast.instance.impl.HazelcastInstanceFactory; +import java.io.IOException; /** This command is used to execute the SeaTunnel engine job by SeaTunnel API. */ public class ServerExecuteCommand implements Command { @@ -37,14 +37,11 @@ public ServerExecuteCommand(ServerCommandArgs serverCommandArgs) { } @Override - public void execute() { + public void execute() throws IOException { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); if (StringUtils.isNotEmpty(serverCommandArgs.getClusterName())) { seaTunnelConfig.getHazelcastConfig().setClusterName(serverCommandArgs.getClusterName()); } - HazelcastInstanceFactory.newHazelcastInstance( - seaTunnelConfig.getHazelcastConfig(), - Thread.currentThread().getName(), - new SeaTunnelNodeContext(seaTunnelConfig)); + SeaTunnelServerStarter.createHazelcastInstance(); } } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java index c68da5faeac..c3b2129c5c3 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -17,10 +17,6 @@ package org.apache.seatunnel.engine.e2e; -import com.hazelcast.client.config.ClientConfig; -import com.hazelcast.instance.impl.HazelcastInstanceImpl; -import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.common.config.DeployMode; import org.apache.seatunnel.engine.client.SeaTunnelClient; @@ -32,19 +28,21 @@ import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; -import org.apache.seatunnel.engine.server.rest.RestConstant; + import org.awaitility.Awaitility; -import org.hamcrest.Matcher; -import org.hamcrest.core.StringContains; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import lombok.extern.slf4j.Slf4j; + +import java.util.concurrent.TimeUnit; + import static io.restassured.RestAssured.given; import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.matchesRegex; @Slf4j @@ -77,59 +75,61 @@ static void beforeClass() throws Exception { clientConfig.setClusterName(testClusterName); SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig); JobExecutionEnvironment jobExecutionEnv = - engineClient.createExecutionContext(filePath, jobConfig); + engineClient.createExecutionContext(filePath, jobConfig); clientJobProxy = jobExecutionEnv.execute(); Awaitility.await() - .atMost(2, TimeUnit.MINUTES) - .untilAsserted( - () -> - Assertions.assertEquals( - JobStatus.RUNNING, clientJobProxy.getJobStatus())); + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + Assertions.assertEquals( + JobStatus.RUNNING, clientJobProxy.getJobStatus())); } @Test - public void testGetMetrics() { - given().get( - HOST - + metricConfig.getHttpPort() - + "/metrics") - .then() - .statusCode(200) - // Use regular expressions to verify whether the response body is the indicator data of Prometheus - // Metric data is usually multi-line, use newlines for validation - .body(matchesRegex("(?s)^.*# HELP.*# TYPE.*$")) - // Verify that the response body contains a specific metric - // JVM metrics - .body(containsString("jvm_threads")) - .body(containsString("jvm_memory_pool")) - .body(containsString("jvm_gc")) - .body(containsString("jvm_info")) - .body(containsString("jvm_memory_bytes")) - .body(containsString("jvm_classes")) - .body(containsString("jvm_buffer_pool")) - .body(containsString("process_start")) - // Thread pool metrics - .body(containsString("st_thread_pool_activeCount{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) - .body(containsString("st_thread_pool_poolSize{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_task_total{address=\"[localhost]:5801\",}")) - // Job count metrics - .body(containsString("st_job_count{type=\"canceled\",} 0.0")) - .body(containsString("st_job_count{type=\"cancelling\",} 0.0")) - .body(containsString("st_job_count{type=\"created\",} 0.0")) - .body(containsString("st_job_count{type=\"failed\",} 0.0")) - .body(containsString("st_job_count{type=\"failing\",} 0.0")) - .body(containsString("st_job_count{type=\"finished\",} 0.0")) - .body(containsString("st_job_count{type=\"reconciling\",} 0.0")) - .body(containsString("st_job_count{type=\"restarting\",} 0.0")) - // Running job count is 1 - .body(containsString("st_job_count{type=\"running\",} 1.0")) - .body(containsString("st_job_count{type=\"scheduled\",} 0.0")) - .body(containsString("st_job_count{type=\"suspended\",} 0.0")); + public void testGetMetrics() throws InterruptedException { + given().get(HOST + metricConfig.getHttpPort() + "/metrics") + .then() + .statusCode(200) + // Use regular expressions to verify whether the response body is the indicator data + // of Prometheus + // Metric data is usually multi-line, use newlines for validation + .body(matchesRegex("(?s)^.*# HELP.*# TYPE.*$")) + // Verify that the response body contains a specific metric + // JVM metrics + .body(containsString("jvm_threads")) + .body(containsString("jvm_memory_pool")) + .body(containsString("jvm_gc")) + .body(containsString("jvm_info")) + .body(containsString("jvm_memory_bytes")) + .body(containsString("jvm_classes")) + .body(containsString("jvm_buffer_pool")) + .body(containsString("process_start")) + // Job thread pool metrics + .body(containsString("job_thread_pool_activeCount{address=\"[localhost]:5801\",}")) + .body( + containsString( + "job_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) + .body(containsString("job_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) + .body( + containsString( + "job_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) + .body(containsString("job_thread_pool_poolSize{address=\"[localhost]:5801\",}")) + .body(containsString("job_thread_pool_task_total{address=\"[localhost]:5801\",}")) + // Job count metrics + .body(containsString("job_count{type=\"canceled\",} 0.0")) + .body(containsString("job_count{type=\"cancelling\",} 0.0")) + .body(containsString("job_count{type=\"created\",} 0.0")) + .body(containsString("job_count{type=\"failed\",} 0.0")) + .body(containsString("job_count{type=\"failing\",} 0.0")) + .body(containsString("job_count{type=\"finished\",} 0.0")) + .body(containsString("job_count{type=\"reconciling\",} 0.0")) + .body(containsString("job_count{type=\"restarting\",} 0.0")) + // Running job count is 1 + .body(containsString("job_count{type=\"running\",} 1.0")) + .body(containsString("job_count{type=\"scheduled\",} 0.0")) + .body(containsString("job_count{type=\"suspended\",} 0.0")); } @AfterAll diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml index c28e9c94d5c..e1048ab83f9 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml @@ -15,6 +15,7 @@ # limitations under the License. # + hazelcast: cluster-name: seatunnel network: @@ -41,3 +42,4 @@ hazelcast: hazelcast.slow.operation.detector.stacktrace.logging.enabled: true hazelcast.logging.type: log4j2 hazelcast.operation.generic.thread.count: 200 +# hazelcast.jet.enabled: true diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 91da8fc362f..51e18cfea5f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -35,10 +35,13 @@ public static void main(String[] args) throws IOException { createHazelcastInstance(); } - public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { + public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) + throws IOException { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); seaTunnelConfig.getHazelcastConfig().setClusterName(clusterName); - return createHazelcastInstance(seaTunnelConfig); + HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); + createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + return hazelcastInstance; } public static HazelcastInstanceImpl createHazelcastInstance( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java index cfff6d012db..5f636dd6c45 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -20,7 +20,7 @@ import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; import com.hazelcast.instance.impl.Node; import io.prometheus.client.CollectorRegistry; @@ -47,9 +47,10 @@ private void start() throws IOException { HTTPServer httpServer = new HTTPServer.Builder().withPort(metricConfig.getHttpPort()).build(); CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; - // + // Job info detail new JobMetricExports(this).register(collectorRegistry); - new ThreadPoolStatusExports(this).register(collectorRegistry); + // Thread pool status + new JobThreadPoolStatusExports(this).register(collectorRegistry); } public Node getNode() { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index 625380ea107..471a32c5aab 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -43,7 +43,7 @@ public List collect() { JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); GaugeMetricFamily metricFamily = new GaugeMetricFamily( - "st_job_count", + "job_count", "The job count of seatunnel cluster ", Collections.singletonList("type")); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java similarity index 84% rename from seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java rename to seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index 1f3c9944e80..c38c34be42d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -29,11 +29,12 @@ import java.util.Collections; import java.util.List; -public class ThreadPoolStatusExports extends AbstractCollector { +public class JobThreadPoolStatusExports extends AbstractCollector { - private static String HELP = "The %s of seatunnel coordinator executors cachedThreadPool"; + private static String HELP = + "The %s of seatunnel coordinator job's executor cached thread pool"; - public ThreadPoolStatusExports(ExportsInstance exportsInstance) { + public JobThreadPoolStatusExports(ExportsInstance exportsInstance) { super(exportsInstance); } @@ -49,7 +50,7 @@ public List collect() { List labelNames = Collections.singletonList("address"); GaugeMetricFamily activeCount = new GaugeMetricFamily( - "st_thread_pool_activeCount", + "job_thread_pool_activeCount", String.format(HELP, "activeCount"), labelNames); activeCount.addMetric(labelValues, threadPoolStatusMetrics.getActiveCount()); @@ -57,7 +58,7 @@ public List collect() { CounterMetricFamily completedTask = new CounterMetricFamily( - "st_thread_pool_completedTask", + "job_thread_pool_completedTask", String.format(HELP, "completedTask"), labelNames); completedTask.addMetric(labelValues, threadPoolStatusMetrics.getCompletedTaskCount()); @@ -65,7 +66,7 @@ public List collect() { GaugeMetricFamily corePoolSize = new GaugeMetricFamily( - "st_thread_pool_corePoolSize", + "job_thread_pool_corePoolSize", String.format(HELP, "activeCount"), labelNames); corePoolSize.addMetric(labelValues, threadPoolStatusMetrics.getCorePoolSize()); @@ -73,7 +74,7 @@ public List collect() { GaugeMetricFamily maximumPoolSize = new GaugeMetricFamily( - "st_thread_pool_maximumPoolSize", + "job_thread_pool_maximumPoolSize", String.format(HELP, "maximumPoolSize"), labelNames); maximumPoolSize.addMetric(labelValues, threadPoolStatusMetrics.getMaximumPoolSize()); @@ -81,13 +82,13 @@ public List collect() { GaugeMetricFamily poolSize = new GaugeMetricFamily( - "st_thread_pool_poolSize", String.format(HELP, "poolSize"), labelNames); + "job_thread_pool_poolSize", String.format(HELP, "poolSize"), labelNames); poolSize.addMetric(labelValues, threadPoolStatusMetrics.getPoolSize()); mfs.add(poolSize); CounterMetricFamily taskCount = new CounterMetricFamily( - "st_thread_pool_task", String.format(HELP, "taskCount"), labelNames); + "job_thread_pool_task", String.format(HELP, "taskCount"), labelNames); taskCount.addMetric(labelValues, threadPoolStatusMetrics.getTaskCount()); mfs.add(taskCount); diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java index 3cb224e69e0..ba65ded1194 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java @@ -31,9 +31,9 @@ import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.internal.serialization.Data; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.net.MalformedURLException; import java.util.Collections; import java.util.concurrent.TimeUnit; @@ -41,7 +41,7 @@ public class CoordinatorServiceTest { @Test - public void testMasterNodeActive() { + public void testMasterNodeActive() throws IOException { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName("CoordinatorServiceTest_testMasterNodeActive")); @@ -85,7 +85,7 @@ public void testMasterNodeActive() { @SuppressWarnings("checkstyle:RegexpSingleline") @Test public void testClearCoordinatorService() - throws MalformedURLException, NoSuchMethodException, InvocationTargetException, + throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { HazelcastInstanceImpl coordinatorServiceTest = SeaTunnelServerStarter.createHazelcastInstance( @@ -141,7 +141,7 @@ public void testClearCoordinatorService() @Test @Disabled("disabled because we can not know") - public void testJobRestoreWhenMasterNodeSwitch() throws InterruptedException { + public void testJobRestoreWhenMasterNodeSwitch() throws InterruptedException, IOException { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName( diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index 2a7c25e0830..b43dc0cf0e3 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ClientCommandArgs; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; @@ -30,8 +31,9 @@ public class SeaTunnelEngineExample { public static void main(String[] args) - throws FileNotFoundException, URISyntaxException, CommandException { - String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; + throws IOException, URISyntaxException, CommandException { + String configurePath = + args.length > 0 ? args[0] : "/examples/stream_fakesource_to_file.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); @@ -39,7 +41,7 @@ public static void main(String[] args) clientCommandArgs.setJobName(Paths.get(configFile).getFileName().toString()); // Change Execution Mode to CLUSTER to use client mode, before do this, you should start // SeaTunnelEngineServerExample - clientCommandArgs.setMasterType(MasterType.LOCAL); + clientCommandArgs.setMasterType(MasterType.CLUSTER); SeaTunnel.run(clientCommandArgs.buildCommand()); } diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java index df539ea6e05..2bb1ed69c4b 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java @@ -21,8 +21,10 @@ import org.apache.seatunnel.core.starter.exception.CommandException; import org.apache.seatunnel.core.starter.seatunnel.args.ServerCommandArgs; +import java.io.IOException; + public class SeaTunnelEngineServerExample { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { ServerCommandArgs serverCommandArgs = new ServerCommandArgs(); SeaTunnel.run(serverCommandArgs.buildCommand()); } diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java index 553c1963622..f6b1c6a7941 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; @@ -29,7 +30,7 @@ public class SeaTunnelApiExample { public static void main(String[] args) - throws FileNotFoundException, URISyntaxException, CommandException { + throws IOException, URISyntaxException, CommandException { String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); FlinkCommandArgs flinkCommandArgs = new FlinkCommandArgs(); diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java index 2c7ddc5de69..0bbfff04356 100644 --- a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java +++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; @@ -30,7 +31,7 @@ public class ExampleUtils { public static void builder(String configurePath) - throws FileNotFoundException, URISyntaxException, CommandException { + throws IOException, URISyntaxException, CommandException { String configFile = getTestConfigFile(configurePath); SparkCommandArgs sparkCommandArgs = new SparkCommandArgs(); sparkCommandArgs.setConfigFile(configFile); diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java index 870ec64895a..b0a12dc3fc4 100644 --- a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java +++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java @@ -19,13 +19,13 @@ import org.apache.seatunnel.core.starter.exception.CommandException; -import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; public class SeaTunnelApiExample { public static void main(String[] args) - throws FileNotFoundException, URISyntaxException, CommandException { + throws IOException, URISyntaxException, CommandException { String configurePath = args.length > 0 ? args[0] : "/examples/spark.batch.conf"; ExampleUtils.builder(configurePath); } From f724d2bc3703e4a4420b2870227b929a766c6ea8 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 18:54:30 +0800 Subject: [PATCH 06/30] [Server-core][telemetry] improving code and docs --- config/seatunnel.yaml | 3 +- .../seatunnel-engine/grafana/dashboard.json | 1904 +++++++++++++++++ .../seatunnel-engine/openmetrics/metrics.txt | 280 +++ .../seatunnel-engine/prometheus/metrics.txt | 275 +++ docs/en/seatunnel-engine/telemetry.md | 167 +- .../command/ServerExecuteCommand.java | 4 +- seatunnel-dist/release-docs/LICENSE | 14 +- .../seatunnel/engine/e2e/JobExecutionIT.java | 2 +- .../seatunnel/engine/e2e/TelemetryApiIT.java | 441 +++- .../src/test/resources/hazelcast.yaml | 1 - .../YamlSeaTunnelDomConfigProcessor.java | 11 +- .../config/server/ServerConfigOptions.java | 12 +- .../config/server/TelemetryMetricConfig.java | 13 +- .../src/main/resources/seatunnel.yaml | 3 +- .../engine/server/NodeExtension.java | 4 + .../engine/server/SeaTunnelServerStarter.java | 31 +- .../engine/server/rest/RestConstant.java | 3 + .../rest/RestHttpGetCommandProcessor.java | 34 + .../telemetry/metrics/AbstractCollector.java | 62 + .../telemetry/metrics/ExportsInstance.java | 35 +- ...y.java => ExportsInstanceInitializer.java} | 13 +- .../metrics/exports/ClusterMetricExports.java | 94 + .../metrics/exports/JobMetricExports.java | 37 +- .../exports/JobThreadPoolStatusExports.java | 9 +- .../metrics/exports/NodeMetricExports.java | 407 ++++ .../engine/server/CoordinatorServiceTest.java | 8 +- 26 files changed, 3670 insertions(+), 197 deletions(-) create mode 100644 docs/en/seatunnel-engine/grafana/dashboard.json create mode 100644 docs/en/seatunnel-engine/openmetrics/metrics.txt create mode 100644 docs/en/seatunnel-engine/prometheus/metrics.txt rename seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/{ExportsInstanceFactory.java => ExportsInstanceInitializer.java} (71%) create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java diff --git a/config/seatunnel.yaml b/config/seatunnel.yaml index 67c42868e31..104126437d2 100644 --- a/config/seatunnel.yaml +++ b/config/seatunnel.yaml @@ -37,5 +37,4 @@ seatunnel: fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission telemetry: metric: - http-port: 9090 - load-default-exports: false \ No newline at end of file + enabled: true \ No newline at end of file diff --git a/docs/en/seatunnel-engine/grafana/dashboard.json b/docs/en/seatunnel-engine/grafana/dashboard.json new file mode 100644 index 00000000000..a0457631055 --- /dev/null +++ b/docs/en/seatunnel-engine/grafana/dashboard.json @@ -0,0 +1,1904 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 0, + "id": 1, + "links": [], + "liveNow": false, + "panels": [ + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-orange", + "value": null + }, + { + "color": "red", + "value": 100000 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 17, + "options": { + "colorMode": "background", + "graphMode": "none", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "8.3.3", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "node_count", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "Total Node Count", + "type": "stat" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 18, + "options": { + "colorMode": "background", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "text": { + "titleSize": 1 + }, + "textMode": "auto" + }, + "pluginVersion": "8.3.3", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "sum(node_state)", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "UP Node Count", + "type": "stat" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 22, + "panels": [], + "title": "Hazelcast Partition", + "type": "row" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 5 + }, + "id": 32, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_partitionCount", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "partitionCount", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 5 + }, + "id": 33, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_activePartition", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "activePartition", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 12 + }, + "id": 34, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "pluginVersion": "8.3.3", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_isClusterSafe", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "isClusterSafe", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 12 + }, + "id": 35, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_isLocalMemberSafe", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "isLocalMemberSafe", + "type": "timeseries" + }, + { + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 19 + }, + "id": 20, + "title": "Hazelcast Executor", + "type": "row" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 24, + "x": 0, + "y": 20 + }, + "id": 24, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_executedCount", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "executedCount", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 26 + }, + "id": 26, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_isTerminated", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "isTerminated", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 26 + }, + "id": 25, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_isShutdown", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "isShutdown", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 34 + }, + "id": 28, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_poolSize", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "poolSize", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 34 + }, + "id": 27, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_maxPoolSize", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "maxPoolSize", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 42 + }, + "id": 30, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_queueRemainingCapacity", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "queueRemainingCapacity", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 42 + }, + "id": 29, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_queueSize", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "queueSize", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 50 + }, + "id": 7, + "panels": [], + "title": "System", + "type": "row" + }, + { + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 51 + }, + "id": 9, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "100 * (1 - rate(process_cpu_seconds_total[15s]))", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "Cpu Usage", + "type": "timeseries" + }, + { + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 51 + }, + "id": 10, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "100 * (jvm_memory_bytes_used{area=\"heap\"} / jvm_memory_bytes_max{area=\"heap\"})", + "interval": "", + "legendFormat": "{{instance}}-{{area}}", + "refId": "A" + } + ], + "title": "Heap Memory Usage", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 59 + }, + "id": 12, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "round(delta(jvm_gc_collection_seconds_count[15s]))", + "interval": "", + "legendFormat": "{{instance}}-{{gc}}", + "refId": "A" + } + ], + "title": "GC Count", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 59 + }, + "id": 13, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "delta(jvm_gc_collection_seconds_sum[15s]) * 1000", + "interval": "", + "legendFormat": "{{instance}}-{{gc}}", + "refId": "A" + } + ], + "title": "GC Cost Time", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 67 + }, + "id": 14, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_current", + "interval": "", + "legendFormat": "{{instance}}-current", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_daemon", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-daemon", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_peak", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-peak", + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_deadlocked", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-deadlocked", + "refId": "D" + } + ], + "title": "Jvm Threads", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 75 + }, + "id": 5, + "panels": [], + "title": "Job", + "type": "row" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 24, + "x": 0, + "y": 76 + }, + "id": 2, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "sum(job_count) by (type) ", + "interval": "", + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "title": "Job Count", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 82 + }, + "id": 3, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_activeCount", + "interval": "", + "legendFormat": "{{instance}}-activeCount", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_corePoolSize", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-corePoolSize", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_maximumPoolSize", + "hide": true, + "interval": "", + "legendFormat": "{{instance}}-maximumPoolSize", + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_poolSize", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-poolSize", + "refId": "D" + } + ], + "title": "Job Thread Pool", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 82 + }, + "id": 15, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_completedTask_total", + "interval": "", + "legendFormat": "{{instance}}-completedTaskTotal", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_task_total", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-taskTotal", + "refId": "B" + } + ], + "title": "Job Thread Pool Total", + "type": "timeseries" + } + ], + "refresh": "5s", + "schemaVersion": 34, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-15m", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "Seatunnel Cluster", + "uid": "BDfcs-CVz", + "version": 11, + "weekStart": "" +} \ No newline at end of file diff --git a/docs/en/seatunnel-engine/openmetrics/metrics.txt b/docs/en/seatunnel-engine/openmetrics/metrics.txt new file mode 100644 index 00000000000..ea1855fb113 --- /dev/null +++ b/docs/en/seatunnel-engine/openmetrics/metrics.txt @@ -0,0 +1,280 @@ +# TYPE job_count gauge +# HELP job_count All job counts of seatunnel cluster +job_count{cluster="seatunnel",type="canceled"} 0.0 +job_count{cluster="seatunnel",type="cancelling"} 0.0 +job_count{cluster="seatunnel",type="created"} 0.0 +job_count{cluster="seatunnel",type="failed"} 0.0 +job_count{cluster="seatunnel",type="failing"} 0.0 +job_count{cluster="seatunnel",type="finished"} 0.0 +job_count{cluster="seatunnel",type="reconciling"} 0.0 +job_count{cluster="seatunnel",type="restarting"} 0.0 +job_count{cluster="seatunnel",type="running"} 0.0 +job_count{cluster="seatunnel",type="scheduled"} 0.0 +job_count{cluster="seatunnel",type="suspended"} 0.0 +# TYPE process_cpu_seconds counter +# HELP process_cpu_seconds Total user and system CPU time spent in seconds. +process_cpu_seconds_total 15.74875 +# TYPE process_start_time_seconds gauge +# HELP process_start_time_seconds Start time of the process since unix epoch in seconds. +process_start_time_seconds 1.689671779061E9 +# TYPE process_open_fds gauge +# HELP process_open_fds Number of open file descriptors. +process_open_fds 112.0 +# TYPE process_max_fds gauge +# HELP process_max_fds Maximum number of open file descriptors. +process_max_fds 10240.0 +# TYPE cluster_info gauge +# HELP cluster_info Cluster info +cluster_info{cluster="seatunnel",hazelcastVersion="5.1",master="127.0.0.1:5801"} 1.0 +# TYPE cluster_time gauge +# HELP cluster_time Cluster start time +cluster_time{cluster="seatunnel",hazelcastVersion="5.1"} 1.68967187357E12 +# TYPE node_count gauge +# HELP node_count Cluster node total count +node_count{cluster="seatunnel"} 1.0 +# TYPE jvm_threads_current gauge +# HELP jvm_threads_current Current thread count of a JVM +jvm_threads_current 105.0 +# TYPE jvm_threads_daemon gauge +# HELP jvm_threads_daemon Daemon thread count of a JVM +jvm_threads_daemon 7.0 +# TYPE jvm_threads_peak gauge +# HELP jvm_threads_peak Peak thread count of a JVM +jvm_threads_peak 118.0 +# TYPE jvm_threads_started counter +# HELP jvm_threads_started Started thread count of a JVM +jvm_threads_started_total 122.0 +# TYPE jvm_threads_deadlocked gauge +# HELP jvm_threads_deadlocked Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers +jvm_threads_deadlocked 0.0 +# TYPE jvm_threads_deadlocked_monitor gauge +# HELP jvm_threads_deadlocked_monitor Cycles of JVM-threads that are in deadlock waiting to acquire object monitors +jvm_threads_deadlocked_monitor 0.0 +# TYPE jvm_threads_state gauge +# HELP jvm_threads_state Current count of threads by state +jvm_threads_state{state="NEW"} 0.0 +jvm_threads_state{state="TERMINATED"} 0.0 +jvm_threads_state{state="RUNNABLE"} 12.0 +jvm_threads_state{state="BLOCKED"} 0.0 +jvm_threads_state{state="WAITING"} 76.0 +jvm_threads_state{state="TIMED_WAITING"} 17.0 +jvm_threads_state{state="UNKNOWN"} 0.0 +# TYPE jvm_classes_currently_loaded gauge +# HELP jvm_classes_currently_loaded The number of classes that are currently loaded in the JVM +jvm_classes_currently_loaded 8317.0 +# TYPE jvm_classes_loaded counter +# HELP jvm_classes_loaded The total number of classes that have been loaded since the JVM has started execution +jvm_classes_loaded_total 8317.0 +# TYPE jvm_classes_unloaded counter +# HELP jvm_classes_unloaded The total number of classes that have been unloaded since the JVM has started execution +jvm_classes_unloaded_total 0.0 +# TYPE node_state gauge +# HELP node_state Whether is up of seatunnel node +node_state{cluster="seatunnel",address="127.0.0.1:5801"} 1.0 +# TYPE hazelcast_executor_executedCount gauge +# HELP hazelcast_executor_executedCount The hazelcast executor executedCount of seatunnel cluster node +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 136.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 1601.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_isShutdown gauge +# HELP hazelcast_executor_isShutdown The hazelcast executor isShutdown of seatunnel cluster node +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_isTerminated gauge +# HELP hazelcast_executor_isTerminated The hazelcast executor isTerminated of seatunnel cluster node +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_maxPoolSize gauge +# HELP hazelcast_executor_maxPoolSize The hazelcast executor maxPoolSize of seatunnel cluster node +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 160.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 8.0 +# TYPE hazelcast_executor_poolSize gauge +# HELP hazelcast_executor_poolSize The hazelcast executor poolSize of seatunnel cluster node +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_queueRemainingCapacity gauge +# HELP hazelcast_executor_queueRemainingCapacity The hazelcast executor queueRemainingCapacity of seatunnel cluster +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 2.147483647E9 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 2.147483647E9 +# TYPE hazelcast_executor_queueSize gauge +# HELP hazelcast_executor_queueSize The hazelcast executor queueSize of seatunnel cluster node +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_partition_partitionCount gauge +# HELP hazelcast_partition_partitionCount The partitionCount of seatunnel cluster node +hazelcast_partition_partitionCount{cluster="seatunnel",address="127.0.0.1:5801"} 271.0 +# TYPE hazelcast_partition_activePartition gauge +# HELP hazelcast_partition_activePartition The activePartition of seatunnel cluster node +hazelcast_partition_activePartition{cluster="seatunnel",address="127.0.0.1:5801"} 271.0 +# TYPE hazelcast_partition_isClusterSafe gauge +# HELP hazelcast_partition_isClusterSafe Whether is cluster safe of partition +hazelcast_partition_isClusterSafe{cluster="seatunnel",address="127.0.0.1:5801"} 1.0 +# TYPE hazelcast_partition_isLocalMemberSafe gauge +# HELP hazelcast_partition_isLocalMemberSafe Whether is local member safe of partition +hazelcast_partition_isLocalMemberSafe{cluster="seatunnel",address="127.0.0.1:5801"} 1.0 +# TYPE jvm_memory_pool_allocated_bytes counter +# HELP jvm_memory_pool_allocated_bytes Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously. +jvm_memory_pool_allocated_bytes_total{pool="Code Cache"} 8118976.0 +jvm_memory_pool_allocated_bytes_created{pool="Code Cache"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="PS Eden Space"} 2.39174704E8 +jvm_memory_pool_allocated_bytes_created{pool="PS Eden Space"} 1.689671786515E9 +jvm_memory_pool_allocated_bytes_total{pool="PS Old Gen"} 1.8928464E7 +jvm_memory_pool_allocated_bytes_created{pool="PS Old Gen"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="PS Survivor Space"} 9945064.0 +jvm_memory_pool_allocated_bytes_created{pool="PS Survivor Space"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="Compressed Class Space"} 4591800.0 +jvm_memory_pool_allocated_bytes_created{pool="Compressed Class Space"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="Metaspace"} 3.2616592E7 +jvm_memory_pool_allocated_bytes_created{pool="Metaspace"} 1.689671786522E9 +# TYPE jvm_buffer_pool_used_bytes gauge +# HELP jvm_buffer_pool_used_bytes Used bytes of a given JVM buffer pool. +jvm_buffer_pool_used_bytes{pool="direct"} 412746.0 +jvm_buffer_pool_used_bytes{pool="mapped"} 0.0 +# TYPE jvm_buffer_pool_capacity_bytes gauge +# HELP jvm_buffer_pool_capacity_bytes Bytes capacity of a given JVM buffer pool. +jvm_buffer_pool_capacity_bytes{pool="direct"} 412743.0 +jvm_buffer_pool_capacity_bytes{pool="mapped"} 0.0 +# TYPE jvm_buffer_pool_used_buffers gauge +# HELP jvm_buffer_pool_used_buffers Used buffers of a given JVM buffer pool. +jvm_buffer_pool_used_buffers{pool="direct"} 7.0 +jvm_buffer_pool_used_buffers{pool="mapped"} 0.0 +# TYPE job_thread_pool_activeCount gauge +# HELP job_thread_pool_activeCount The activeCount of seatunnel coordinator job's executor cached thread pool +job_thread_pool_activeCount{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_completedTask counter +# HELP job_thread_pool_completedTask The completedTask of seatunnel coordinator job's executor cached thread pool +job_thread_pool_completedTask_total{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_corePoolSize gauge +# HELP job_thread_pool_corePoolSize The corePoolSize of seatunnel coordinator job's executor cached thread pool +job_thread_pool_corePoolSize{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_maximumPoolSize gauge +# HELP job_thread_pool_maximumPoolSize The maximumPoolSize of seatunnel coordinator job's executor cached thread pool +job_thread_pool_maximumPoolSize{cluster="seatunnel",address="127.0.0.1:5801"} 2.147483647E9 +# TYPE job_thread_pool_poolSize gauge +# HELP job_thread_pool_poolSize The poolSize of seatunnel coordinator job's executor cached thread pool +job_thread_pool_poolSize{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_task counter +# HELP job_thread_pool_task The taskCount of seatunnel coordinator job's executor cached thread pool +job_thread_pool_task_total{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE jvm_gc_collection_seconds summary +# HELP jvm_gc_collection_seconds Time spent in a given JVM garbage collector in seconds. +jvm_gc_collection_seconds_count{gc="PS Scavenge"} 6.0 +jvm_gc_collection_seconds_sum{gc="PS Scavenge"} 0.161 +jvm_gc_collection_seconds_count{gc="PS MarkSweep"} 2.0 +jvm_gc_collection_seconds_sum{gc="PS MarkSweep"} 0.068 +# TYPE jvm_memory_objects_pending_finalization gauge +# HELP jvm_memory_objects_pending_finalization The number of objects waiting in the finalizer queue. +jvm_memory_objects_pending_finalization 0.0 +# TYPE jvm_memory_bytes_used gauge +# HELP jvm_memory_bytes_used Used bytes of a given JVM memory area. +jvm_memory_bytes_used{area="heap"} 2.72221784E8 +jvm_memory_bytes_used{area="nonheap"} 5.6619408E7 +# TYPE jvm_memory_bytes_committed gauge +# HELP jvm_memory_bytes_committed Committed (bytes) of a given JVM memory area. +jvm_memory_bytes_committed{area="heap"} 6.13941248E8 +jvm_memory_bytes_committed{area="nonheap"} 6.2537728E7 +# TYPE jvm_memory_bytes_max gauge +# HELP jvm_memory_bytes_max Max (bytes) of a given JVM memory area. +jvm_memory_bytes_max{area="heap"} 3.817865216E9 +jvm_memory_bytes_max{area="nonheap"} -1.0 +# TYPE jvm_memory_bytes_init gauge +# HELP jvm_memory_bytes_init Initial bytes of a given JVM memory area. +jvm_memory_bytes_init{area="heap"} 2.68435456E8 +jvm_memory_bytes_init{area="nonheap"} 2555904.0 +# TYPE jvm_memory_pool_bytes_used gauge +# HELP jvm_memory_pool_bytes_used Used bytes of a given JVM memory pool. +jvm_memory_pool_bytes_used{pool="Code Cache"} 1.0232064E7 +jvm_memory_pool_bytes_used{pool="Metaspace"} 4.0859424E7 +jvm_memory_pool_bytes_used{pool="Compressed Class Space"} 5527920.0 +jvm_memory_pool_bytes_used{pool="PS Eden Space"} 2.5329332E8 +jvm_memory_pool_bytes_used{pool="PS Survivor Space"} 0.0 +jvm_memory_pool_bytes_used{pool="PS Old Gen"} 1.8928464E7 +# TYPE jvm_memory_pool_bytes_committed gauge +# HELP jvm_memory_pool_bytes_committed Committed bytes of a given JVM memory pool. +jvm_memory_pool_bytes_committed{pool="Code Cache"} 1.081344E7 +jvm_memory_pool_bytes_committed{pool="Metaspace"} 4.52608E7 +jvm_memory_pool_bytes_committed{pool="Compressed Class Space"} 6463488.0 +jvm_memory_pool_bytes_committed{pool="PS Eden Space"} 4.90733568E8 +jvm_memory_pool_bytes_committed{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_bytes_committed{pool="PS Old Gen"} 1.13246208E8 +# TYPE jvm_memory_pool_bytes_max gauge +# HELP jvm_memory_pool_bytes_max Max bytes of a given JVM memory pool. +jvm_memory_pool_bytes_max{pool="Code Cache"} 2.5165824E8 +jvm_memory_pool_bytes_max{pool="Metaspace"} -1.0 +jvm_memory_pool_bytes_max{pool="Compressed Class Space"} 1.073741824E9 +jvm_memory_pool_bytes_max{pool="PS Eden Space"} 1.40771328E9 +jvm_memory_pool_bytes_max{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_bytes_max{pool="PS Old Gen"} 2.863661056E9 +# TYPE jvm_memory_pool_bytes_init gauge +# HELP jvm_memory_pool_bytes_init Initial bytes of a given JVM memory pool. +jvm_memory_pool_bytes_init{pool="Code Cache"} 2555904.0 +jvm_memory_pool_bytes_init{pool="Metaspace"} 0.0 +jvm_memory_pool_bytes_init{pool="Compressed Class Space"} 0.0 +jvm_memory_pool_bytes_init{pool="PS Eden Space"} 6.7108864E7 +jvm_memory_pool_bytes_init{pool="PS Survivor Space"} 1.1010048E7 +jvm_memory_pool_bytes_init{pool="PS Old Gen"} 1.79306496E8 +# TYPE jvm_memory_pool_collection_used_bytes gauge +# HELP jvm_memory_pool_collection_used_bytes Used bytes after last collection of a given JVM memory pool. +jvm_memory_pool_collection_used_bytes{pool="PS Eden Space"} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Survivor Space"} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Old Gen"} 1.8928464E7 +# TYPE jvm_memory_pool_collection_committed_bytes gauge +# HELP jvm_memory_pool_collection_committed_bytes Committed after last collection bytes of a given JVM memory pool. +jvm_memory_pool_collection_committed_bytes{pool="PS Eden Space"} 4.90733568E8 +jvm_memory_pool_collection_committed_bytes{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_collection_committed_bytes{pool="PS Old Gen"} 1.13246208E8 +# TYPE jvm_memory_pool_collection_max_bytes gauge +# HELP jvm_memory_pool_collection_max_bytes Max bytes after last collection of a given JVM memory pool. +jvm_memory_pool_collection_max_bytes{pool="PS Eden Space"} 1.40771328E9 +jvm_memory_pool_collection_max_bytes{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_collection_max_bytes{pool="PS Old Gen"} 2.863661056E9 +# TYPE jvm_memory_pool_collection_init_bytes gauge +# HELP jvm_memory_pool_collection_init_bytes Initial after last collection bytes of a given JVM memory pool. +jvm_memory_pool_collection_init_bytes{pool="PS Eden Space"} 6.7108864E7 +jvm_memory_pool_collection_init_bytes{pool="PS Survivor Space"} 1.1010048E7 +jvm_memory_pool_collection_init_bytes{pool="PS Old Gen"} 1.79306496E8 +# TYPE jvm info +# HELP jvm VM version info +jvm_info{runtime="Java(TM) SE Runtime Environment",vendor="Oracle Corporation",version="1.8.0_212-b10"} 1.0 +# EOF \ No newline at end of file diff --git a/docs/en/seatunnel-engine/prometheus/metrics.txt b/docs/en/seatunnel-engine/prometheus/metrics.txt new file mode 100644 index 00000000000..e43dc6934a5 --- /dev/null +++ b/docs/en/seatunnel-engine/prometheus/metrics.txt @@ -0,0 +1,275 @@ +# HELP job_count All job counts of seatunnel cluster +# TYPE job_count gauge +job_count{cluster="seatunnel",type="canceled",} 0.0 +job_count{cluster="seatunnel",type="cancelling",} 0.0 +job_count{cluster="seatunnel",type="created",} 0.0 +job_count{cluster="seatunnel",type="failed",} 0.0 +job_count{cluster="seatunnel",type="failing",} 0.0 +job_count{cluster="seatunnel",type="finished",} 0.0 +job_count{cluster="seatunnel",type="reconciling",} 0.0 +job_count{cluster="seatunnel",type="restarting",} 0.0 +job_count{cluster="seatunnel",type="running",} 0.0 +job_count{cluster="seatunnel",type="scheduled",} 0.0 +job_count{cluster="seatunnel",type="suspended",} 0.0 +# HELP jvm_classes_currently_loaded The number of classes that are currently loaded in the JVM +# TYPE jvm_classes_currently_loaded gauge +jvm_classes_currently_loaded 8186.0 +# HELP jvm_classes_loaded_total The total number of classes that have been loaded since the JVM has started execution +# TYPE jvm_classes_loaded_total counter +jvm_classes_loaded_total 8186.0 +# HELP jvm_classes_unloaded_total The total number of classes that have been unloaded since the JVM has started execution +# TYPE jvm_classes_unloaded_total counter +jvm_classes_unloaded_total 0.0 +# HELP job_thread_pool_activeCount The activeCount of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_activeCount gauge +job_thread_pool_activeCount{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_completedTask_total The completedTask of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_completedTask_total counter +job_thread_pool_completedTask_total{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_corePoolSize The corePoolSize of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_corePoolSize gauge +job_thread_pool_corePoolSize{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_maximumPoolSize The maximumPoolSize of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_maximumPoolSize gauge +job_thread_pool_maximumPoolSize{cluster="seatunnel",address="[localhost]:5801",} 2.147483647E9 +# HELP job_thread_pool_poolSize The poolSize of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_poolSize gauge +job_thread_pool_poolSize{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_task_total The taskCount of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_task_total counter +job_thread_pool_task_total{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP jvm_info VM version info +# TYPE jvm_info gauge +jvm_info{runtime="Java(TM) SE Runtime Environment",vendor="Oracle Corporation",version="1.8.0_212-b10",} 1.0 +# HELP jvm_buffer_pool_used_bytes Used bytes of a given JVM buffer pool. +# TYPE jvm_buffer_pool_used_bytes gauge +jvm_buffer_pool_used_bytes{pool="direct",} 150449.0 +jvm_buffer_pool_used_bytes{pool="mapped",} 0.0 +# HELP jvm_buffer_pool_capacity_bytes Bytes capacity of a given JVM buffer pool. +# TYPE jvm_buffer_pool_capacity_bytes gauge +jvm_buffer_pool_capacity_bytes{pool="direct",} 150447.0 +jvm_buffer_pool_capacity_bytes{pool="mapped",} 0.0 +# HELP jvm_buffer_pool_used_buffers Used buffers of a given JVM buffer pool. +# TYPE jvm_buffer_pool_used_buffers gauge +jvm_buffer_pool_used_buffers{pool="direct",} 5.0 +jvm_buffer_pool_used_buffers{pool="mapped",} 0.0 +# HELP jvm_memory_pool_allocated_bytes_total Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously. +# TYPE jvm_memory_pool_allocated_bytes_total counter +jvm_memory_pool_allocated_bytes_total{pool="Code Cache",} 8709504.0 +jvm_memory_pool_allocated_bytes_total{pool="PS Eden Space",} 4.9551024E7 +jvm_memory_pool_allocated_bytes_total{pool="PS Old Gen",} 1.1479696E7 +jvm_memory_pool_allocated_bytes_total{pool="PS Survivor Space",} 8889712.0 +jvm_memory_pool_allocated_bytes_total{pool="Compressed Class Space",} 4552568.0 +jvm_memory_pool_allocated_bytes_total{pool="Metaspace",} 3.256092E7 +# HELP jvm_gc_collection_seconds Time spent in a given JVM garbage collector in seconds. +# TYPE jvm_gc_collection_seconds summary +jvm_gc_collection_seconds_count{gc="PS Scavenge",} 8.0 +jvm_gc_collection_seconds_sum{gc="PS Scavenge",} 0.117 +jvm_gc_collection_seconds_count{gc="PS MarkSweep",} 2.0 +jvm_gc_collection_seconds_sum{gc="PS MarkSweep",} 0.071 +# HELP process_cpu_seconds_total Total user and system CPU time spent in seconds. +# TYPE process_cpu_seconds_total counter +process_cpu_seconds_total 13.922742 +# HELP process_start_time_seconds Start time of the process since unix epoch in seconds. +# TYPE process_start_time_seconds gauge +process_start_time_seconds 1.689662228826E9 +# HELP process_open_fds Number of open file descriptors. +# TYPE process_open_fds gauge +process_open_fds 113.0 +# HELP process_max_fds Maximum number of open file descriptors. +# TYPE process_max_fds gauge +process_max_fds 10240.0 +# HELP jvm_memory_objects_pending_finalization The number of objects waiting in the finalizer queue. +# TYPE jvm_memory_objects_pending_finalization gauge +jvm_memory_objects_pending_finalization 0.0 +# HELP jvm_memory_bytes_used Used bytes of a given JVM memory area. +# TYPE jvm_memory_bytes_used gauge +jvm_memory_bytes_used{area="heap",} 2.10046672E8 +jvm_memory_bytes_used{area="nonheap",} 5.7037864E7 +# HELP jvm_memory_bytes_committed Committed (bytes) of a given JVM memory area. +# TYPE jvm_memory_bytes_committed gauge +jvm_memory_bytes_committed{area="heap",} 6.01358336E8 +jvm_memory_bytes_committed{area="nonheap",} 6.221824E7 +# HELP jvm_memory_bytes_max Max (bytes) of a given JVM memory area. +# TYPE jvm_memory_bytes_max gauge +jvm_memory_bytes_max{area="heap",} 3.817865216E9 +jvm_memory_bytes_max{area="nonheap",} -1.0 +# HELP jvm_memory_bytes_init Initial bytes of a given JVM memory area. +# TYPE jvm_memory_bytes_init gauge +jvm_memory_bytes_init{area="heap",} 2.68435456E8 +jvm_memory_bytes_init{area="nonheap",} 2555904.0 +# HELP jvm_memory_pool_bytes_used Used bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_used gauge +jvm_memory_pool_bytes_used{pool="Code Cache",} 1.0777728E7 +jvm_memory_pool_bytes_used{pool="Metaspace",} 4.080908E7 +jvm_memory_pool_bytes_used{pool="Compressed Class Space",} 5451056.0 +jvm_memory_pool_bytes_used{pool="PS Eden Space",} 1.98927776E8 +jvm_memory_pool_bytes_used{pool="PS Survivor Space",} 0.0 +jvm_memory_pool_bytes_used{pool="PS Old Gen",} 1.1118896E7 +# HELP jvm_memory_pool_bytes_committed Committed bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_committed gauge +jvm_memory_pool_bytes_committed{pool="Code Cache",} 1.1010048E7 +jvm_memory_pool_bytes_committed{pool="Metaspace",} 4.4867584E7 +jvm_memory_pool_bytes_committed{pool="Compressed Class Space",} 6340608.0 +jvm_memory_pool_bytes_committed{pool="PS Eden Space",} 4.9283072E8 +jvm_memory_pool_bytes_committed{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_bytes_committed{pool="PS Old Gen",} 1.02760448E8 +# HELP jvm_memory_pool_bytes_max Max bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_max gauge +jvm_memory_pool_bytes_max{pool="Code Cache",} 2.5165824E8 +jvm_memory_pool_bytes_max{pool="Metaspace",} -1.0 +jvm_memory_pool_bytes_max{pool="Compressed Class Space",} 1.073741824E9 +jvm_memory_pool_bytes_max{pool="PS Eden Space",} 1.414004736E9 +jvm_memory_pool_bytes_max{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_bytes_max{pool="PS Old Gen",} 2.863661056E9 +# HELP jvm_memory_pool_bytes_init Initial bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_init gauge +jvm_memory_pool_bytes_init{pool="Code Cache",} 2555904.0 +jvm_memory_pool_bytes_init{pool="Metaspace",} 0.0 +jvm_memory_pool_bytes_init{pool="Compressed Class Space",} 0.0 +jvm_memory_pool_bytes_init{pool="PS Eden Space",} 6.7108864E7 +jvm_memory_pool_bytes_init{pool="PS Survivor Space",} 1.1010048E7 +jvm_memory_pool_bytes_init{pool="PS Old Gen",} 1.79306496E8 +# HELP jvm_memory_pool_collection_used_bytes Used bytes after last collection of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_used_bytes gauge +jvm_memory_pool_collection_used_bytes{pool="PS Eden Space",} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Survivor Space",} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Old Gen",} 1.1118896E7 +# HELP jvm_memory_pool_collection_committed_bytes Committed after last collection bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_committed_bytes gauge +jvm_memory_pool_collection_committed_bytes{pool="PS Eden Space",} 4.9283072E8 +jvm_memory_pool_collection_committed_bytes{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_collection_committed_bytes{pool="PS Old Gen",} 1.02760448E8 +# HELP jvm_memory_pool_collection_max_bytes Max bytes after last collection of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_max_bytes gauge +jvm_memory_pool_collection_max_bytes{pool="PS Eden Space",} 1.414004736E9 +jvm_memory_pool_collection_max_bytes{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_collection_max_bytes{pool="PS Old Gen",} 2.863661056E9 +# HELP jvm_memory_pool_collection_init_bytes Initial after last collection bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_init_bytes gauge +jvm_memory_pool_collection_init_bytes{pool="PS Eden Space",} 6.7108864E7 +jvm_memory_pool_collection_init_bytes{pool="PS Survivor Space",} 1.1010048E7 +jvm_memory_pool_collection_init_bytes{pool="PS Old Gen",} 1.79306496E8 +# HELP node_state Whether is up of seatunnel node +# TYPE node_state gauge +node_state{cluster="seatunnel",address="[localhost]:5801",} 1.0 +# HELP node_count Cluster node total count +# TYPE node_count gauge +node_count{cluster="seatunnel",} 1.0 +# HELP hazelcast_executor_executedCount The hazelcast executor executedCount of seatunnel cluster node +# TYPE hazelcast_executor_executedCount gauge +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="io",} 128.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 402.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_isShutdown The hazelcast executor isShutdown of seatunnel cluster node +# TYPE hazelcast_executor_isShutdown gauge +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_isTerminated The hazelcast executor isTerminated of seatunnel cluster node +# TYPE hazelcast_executor_isTerminated gauge +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_maxPoolSize The hazelcast executor maxPoolSize of seatunnel cluster node +# TYPE hazelcast_executor_maxPoolSize gauge +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="async",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="client",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 160.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="io",} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="system",} 8.0 +# HELP hazelcast_executor_poolSize The hazelcast executor poolSize of seatunnel cluster node +# TYPE hazelcast_executor_poolSize gauge +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_queueRemainingCapacity The hazelcast executor queueRemainingCapacity of seatunnel cluster +# TYPE hazelcast_executor_queueRemainingCapacity gauge +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="async",} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="client",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="io",} 2.147483647E9 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="system",} 2.147483647E9 +# HELP hazelcast_executor_queueSize The hazelcast executor queueSize of seatunnel cluster node +# TYPE hazelcast_executor_queueSize gauge +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_partition_partitionCount The partitionCount of seatunnel cluster node +# TYPE hazelcast_partition_partitionCount gauge +hazelcast_partition_partitionCount{cluster="seatunnel",address="[localhost]:5801",} 271.0 +# HELP hazelcast_partition_activePartition The activePartition of seatunnel cluster node +# TYPE hazelcast_partition_activePartition gauge +hazelcast_partition_activePartition{cluster="seatunnel",address="[localhost]:5801",} 271.0 +# HELP hazelcast_partition_isClusterSafe Whether is cluster safe of partition +# TYPE hazelcast_partition_isClusterSafe gauge +hazelcast_partition_isClusterSafe{cluster="seatunnel",address="[localhost]:5801",} 1.0 +# HELP hazelcast_partition_isLocalMemberSafe Whether is local member safe of partition +# TYPE hazelcast_partition_isLocalMemberSafe gauge +hazelcast_partition_isLocalMemberSafe{cluster="seatunnel",address="[localhost]:5801",} 1.0 +# HELP jvm_threads_current Current thread count of a JVM +# TYPE jvm_threads_current gauge +jvm_threads_current 114.0 +# HELP jvm_threads_daemon Daemon thread count of a JVM +# TYPE jvm_threads_daemon gauge +jvm_threads_daemon 5.0 +# HELP jvm_threads_peak Peak thread count of a JVM +# TYPE jvm_threads_peak gauge +jvm_threads_peak 116.0 +# HELP jvm_threads_started_total Started thread count of a JVM +# TYPE jvm_threads_started_total counter +jvm_threads_started_total 119.0 +# HELP jvm_threads_deadlocked Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers +# TYPE jvm_threads_deadlocked gauge +jvm_threads_deadlocked 0.0 +# HELP jvm_threads_deadlocked_monitor Cycles of JVM-threads that are in deadlock waiting to acquire object monitors +# TYPE jvm_threads_deadlocked_monitor gauge +jvm_threads_deadlocked_monitor 0.0 +# HELP jvm_threads_state Current count of threads by state +# TYPE jvm_threads_state gauge +jvm_threads_state{state="NEW",} 0.0 +jvm_threads_state{state="TERMINATED",} 0.0 +jvm_threads_state{state="RUNNABLE",} 10.0 +jvm_threads_state{state="BLOCKED",} 0.0 +jvm_threads_state{state="WAITING",} 75.0 +jvm_threads_state{state="TIMED_WAITING",} 29.0 +jvm_threads_state{state="UNKNOWN",} 0.0 +# HELP jvm_memory_pool_allocated_bytes_created Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously. +# TYPE jvm_memory_pool_allocated_bytes_created gauge +jvm_memory_pool_allocated_bytes_created{pool="Code Cache",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="PS Eden Space",} 1.689662236757E9 +jvm_memory_pool_allocated_bytes_created{pool="PS Old Gen",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="PS Survivor Space",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="Compressed Class Space",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="Metaspace",} 1.689662236806E9 \ No newline at end of file diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index c880c26f8bb..989e8ac0b80 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -15,71 +15,132 @@ The following is an example declarative configuration. seatunnel: engine: telemetry: - http-port: 9090 # The port exposed by the telemetry server, default is 9090. - load-default-exports: true # Whether to load default jvm exports, default is true. + metric: + enabled: true # Whether open metrics export ``` ## Metrics +The [metric text of prometheus](./prometheus/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/metrics`. + +The [metric text of openMetrics](./openmetrics/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/openmetrics`. + Available metrics include the following categories. +Note: All metrics both have the same labelName `cluster`, that's value is the config of `hazelcast.cluster-name`. + +### Node Metrics + +| MetricName | Type | Labels | DESCRIPTION | +|-------------------------------------------|-------|------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------| +| cluster_info | Gauge | **hazelcastVersion**, hazelcastVersion
,*master**, seatunnel master address | Cluster info | +| cluster_time | Gauge | **hazelcastVersion**, hazelcastVersion
,*version**, seatunnel version | Cluster time | +| node_count | Gauge | - | Cluster node total count | +| node_state | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | Whether is up of seatunnel node | +| hazelcast_executor_executedCount | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor executedCount of seatunnel cluster node | +| hazelcast_executor_isShutdown | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor isShutdown of seatunnel cluster node | +| hazelcast_executor_isTerminated | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor isTerminated of seatunnel cluster node | +| hazelcast_executor_maxPoolSize | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor maxPoolSize of seatunnel cluster node | +| hazelcast_executor_poolSize | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor poolSize of seatunnel cluster node | +| hazelcast_executor_queueRemainingCapacity | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor queueRemainingCapacity of seatunnel cluster node | +| hazelcast_executor_queueSize | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor queueSize of seatunnel cluster node | +| hazelcast_partition_partitionCount | Gauge | - | The partitionCount of seatunnel cluster node | +| hazelcast_partition_activePartition | Gauge | - | The activePartition of seatunnel cluster node | +| hazelcast_partition_isClusterSafe | Gauge | - | Weather is cluster safe of partition | +| hazelcast_partition_isLocalMemberSafe | Gauge | - | Weather is local member safe of partition | + ### Thread Pool Status -| MetricName | Type | Labels | -|-------------------------------------|---------|----------------------------------------------------------------------| -| job_thread_pool_activeCount | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_corePoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_maximumPoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_poolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_completedTask_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_task_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | +| MetricName | Type | Labels | DESCRIPTION | +|-------------------------------------|---------|--------------------------------------------------------------------|--------------------------------------------------------------------------------| +| job_thread_pool_activeCount | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The activeCount of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_corePoolSize | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The corePoolSize of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_maximumPoolSize | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The maximumPoolSize of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_poolSize | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The poolSize of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_completedTask_total | Counter | **address**, server instance address,for example: "127.0.0.1:5801" | The completedTask of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_task_total | Counter | **address**, server instance address,for example: "127.0.0.1:5801" | The taskCount of seatunnel coordinator job's executor cached thread pool | ### Job info detail -| MetricName | Type | Labels | -|------------|-------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| job_count | Gauge | **type**, the type of job, including: "canceled" "cancelling" "created" "failed" "failing" "finished" "reconciling" "restarting" "running" "scheduled" "suspended" | +| MetricName | Type | Labels | DESCRIPTION | +|------------|-------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------| +| job_count | Gauge | **type**, the type of job, including: "canceled" "cancelling" "created" "failed" "failing" "finished" "reconciling" "restarting" "running" "scheduled" "suspended" | All job counts of seatunnel cluster | ### JVM Metrics -| MetricName | Type | Labels | -|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------| -| jvm_threads_current | Gauge | - | -| jvm_threads_daemon | Gauge | - | -| jvm_threads_daemon | Gauge | - | -| jvm_threads_peak | Gauge | - | -| jvm_threads_started_total | Counter | - | -| jvm_threads_deadlocked | Gauge | - | -| jvm_threads_deadlocked_monitor | Gauge | - | -| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | -| jvm_classes_currently_loaded | Gauge | - | -| jvm_classes_loaded_total | Counter | - | -| jvm_classes_unloaded_total | Counter | - | -| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | -| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | -| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | -| process_cpu_seconds_total | Counter | - | -| process_start_time_seconds | Gauge | - | -| process_open_fds | Gauge | - | -| process_max_fds | Gauge | - | -| jvm_memory_objects_pending_finalization | Gauge | - | -| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | -| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | -| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | -| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | -| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | -| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | -| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | - -### 实现方案 +| MetricName | Type | Labels | DESCRIPTION | +|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------| +| jvm_threads_current | Gauge | - | Current thread count of a JVM | +| jvm_threads_daemon | Gauge | - | Daemon thread count of a JVM | +| jvm_threads_peak | Gauge | - | Peak thread count of a JVM | +| jvm_threads_started_total | Counter | - | Started thread count of a JVM | +| jvm_threads_deadlocked | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers | +| jvm_threads_deadlocked_monitor | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors | +| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | Current count of threads by state | +| jvm_classes_currently_loaded | Gauge | - | The number of classes that are currently loaded in the JVM | +| jvm_classes_loaded_total | Counter | - | The total number of classes that have been loaded since the JVM has started execution | +| jvm_classes_unloaded_total | Counter | - | The total number of classes that have been unloaded since the JVM has started execution | +| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | VM version info | +| process_cpu_seconds_total | Counter | - | Total user and system CPU time spent in seconds | +| process_start_time_seconds | Gauge | - | Start time of the process since unix epoch in seconds | +| process_open_fds | Gauge | - | Number of open file descriptors | +| process_max_fds | Gauge | - | Maximum number of open file descriptors | +| jvm_memory_objects_pending_finalization | Gauge | - | The number of objects waiting in the finalizer queue | +| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | Used bytes of a given JVM memory area | +| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | Committed (bytes) of a given JVM memory area | +| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | Max (bytes) of a given JVM memory area | +| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | Initial bytes of a given JVM memory area | +| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Used bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Committed bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Max bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Initial bytes of a given JVM memory pool | +| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Used bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Committed after last collection bytes of a given JVM memory pool | +| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Max bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Initial after last collection bytes of a given JVM memory pool | +| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | Used bytes of a given JVM buffer pool | +| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | Bytes capacity of a given JVM buffer pool | +| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | Used buffers of a given JVM buffer pool | + +### Cluster Monitoring By Prometheus & Grafana + +#### Install Prometheus + +For a guide on how to set up Prometheus server go to the [Installation](https://prometheus.io/docs/prometheus/latest/installation) + +#### Configuration Prometheus + +Add seatunnel instance metric exports into `/etc/prometheus/prometheus.yaml`. For example: + +```yaml +global: + # How frequently to scrape targets from this job. + scrape_interval: 15s + +scrape_configs: + # The job name assigned to scraped metrics by default. + - job_name: 'seatunnel' + scrape_interval: 5s + # List of labeled statically configured targets for this job. + static_configs: + # The targets specified by the static config. + - targets: ['localhost:5701'] + # Labels assigned to all metrics scraped from the targets. + # labels: [:] +``` + +#### Install Grafana + +For a guide on how to set up Grafana server go to the [Installation](https://grafana.com/docs/grafana/latest/setup-grafana/installation) + +#### Monitoring Dashboard + +- Add Prometheus DataSource on Grafana. +- Import `Seatunnel Cluster` monitoring dashboard by [Dashboard JSON](./grafana/grafanaDashboard.json) into Grafana. + +The [effect image](../images/cluster_monitoring.png) of the dashboard diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java index e5d11fc3b5c..ec3872e449f 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java @@ -25,8 +25,6 @@ import org.apache.commons.lang3.StringUtils; -import java.io.IOException; - /** This command is used to execute the SeaTunnel engine job by SeaTunnel API. */ public class ServerExecuteCommand implements Command { @@ -37,7 +35,7 @@ public ServerExecuteCommand(ServerCommandArgs serverCommandArgs) { } @Override - public void execute() throws IOException { + public void execute() { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); if (StringUtils.isNotEmpty(serverCommandArgs.getClusterName())) { seaTunnelConfig.getHazelcastConfig().setClusterName(serverCommandArgs.getClusterName()); diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index dca007e2f27..61b5cec0442 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -275,13 +275,13 @@ The text of each license is the standard Apache 2.0 license. (Apache-2.0) listenablefuture (com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava https://mvnrepository.com/artifact/com.google.guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava) (Apache-2.0) accessors-smart (com.google.guava:accessors-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/accessors-smart) (Apache-2.0) json-smart (net.minidev:json-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/json-smart) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient (io.prometheus:simpleclient:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Common (io.prometheus:simpleclient_common:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Hotspot (io.prometheus:simpleclient_hotspot:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Httpserver (io.prometheus:simpleclient_httpserver:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - Common (io.prometheus:simpleclient_tracer_common:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry (io.prometheus:simpleclient_tracer_otel:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry Agent (io.prometheus:simpleclient_tracer_otel_agent:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient (io.prometheus:simpleclient:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Common (io.prometheus:simpleclient_common:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_common/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Hotspot (io.prometheus:simpleclient_hotspot:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_hotspot/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Httpserver (io.prometheus:simpleclient_httpserver:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_httpserver/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - Common (io.prometheus:simpleclient_tracer_common:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_tracer_common/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry (io.prometheus:simpleclient_tracer_otel:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_tracer_otel/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry Agent (io.prometheus:simpleclient_tracer_otel_agent:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_tracer_otel_agent/0.16.0) ======================================================================== MOZILLA PUBLIC LICENSE License diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java index 4609a10dc4c..acd00a8b1a7 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java @@ -46,7 +46,7 @@ public class JobExecutionIT { private static HazelcastInstanceImpl hazelcastInstance; @BeforeAll - public static void beforeClass() throws Exception { + public static void beforeClass() { hazelcastInstance = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName("JobExecutionIT")); diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java index c3b2129c5c3..030d471fa5c 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.JobConfig; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; @@ -54,18 +53,16 @@ public class TelemetryApiIT { private static HazelcastInstanceImpl hazelcastInstance; - private static TelemetryMetricConfig metricConfig; + private static String testClusterName; @BeforeAll static void beforeClass() throws Exception { - String testClusterName = TestUtils.getClusterName("TelemetryApiIT"); + testClusterName = TestUtils.getClusterName("TelemetryApiIT"); SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName); - // get TelemetryMetricConfig - metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); hazelcastInstance = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); // createTelemetryInstance - SeaTunnelServerStarter.createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + SeaTunnelServerStarter.initTelemetryInstance(hazelcastInstance.node); Common.setDeployMode(DeployMode.CLIENT); String filePath = TestUtils.getResource("stream_fakesource_to_file.conf"); JobConfig jobConfig = new JobConfig(); @@ -89,7 +86,14 @@ static void beforeClass() throws Exception { @Test public void testGetMetrics() throws InterruptedException { - given().get(HOST + metricConfig.getHttpPort() + "/metrics") + given().get( + HOST + + hazelcastInstance + .getCluster() + .getLocalMember() + .getAddress() + .getPort() + + "/hazelcast/rest/instance/metrics") .then() .statusCode(200) // Use regular expressions to verify whether the response body is the indicator data @@ -106,30 +110,419 @@ public void testGetMetrics() throws InterruptedException { .body(containsString("jvm_classes")) .body(containsString("jvm_buffer_pool")) .body(containsString("process_start")) + // + .body(containsString("cluster_info{cluster=\"" + testClusterName)) + // + .body(containsString("cluster_time{cluster=\"" + testClusterName)) // Job thread pool metrics - .body(containsString("job_thread_pool_activeCount{address=\"[localhost]:5801\",}")) .body( containsString( - "job_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) - .body(containsString("job_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) + "job_thread_pool_activeCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) .body( containsString( - "job_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) - .body(containsString("job_thread_pool_poolSize{address=\"[localhost]:5801\",}")) - .body(containsString("job_thread_pool_task_total{address=\"[localhost]:5801\",}")) + "job_thread_pool_completedTask_total{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + .body( + containsString( + "job_thread_pool_corePoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + .body( + containsString( + "job_thread_pool_maximumPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",} 2.147483647E9")) + .body( + containsString( + "job_thread_pool_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + .body( + containsString( + "job_thread_pool_task_total{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) // Job count metrics - .body(containsString("job_count{type=\"canceled\",} 0.0")) - .body(containsString("job_count{type=\"cancelling\",} 0.0")) - .body(containsString("job_count{type=\"created\",} 0.0")) - .body(containsString("job_count{type=\"failed\",} 0.0")) - .body(containsString("job_count{type=\"failing\",} 0.0")) - .body(containsString("job_count{type=\"finished\",} 0.0")) - .body(containsString("job_count{type=\"reconciling\",} 0.0")) - .body(containsString("job_count{type=\"restarting\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"canceled\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"cancelling\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"created\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"failed\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"failing\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"finished\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"reconciling\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"restarting\",} 0.0")) // Running job count is 1 - .body(containsString("job_count{type=\"running\",} 1.0")) - .body(containsString("job_count{type=\"scheduled\",} 0.0")) - .body(containsString("job_count{type=\"suspended\",} 0.0")); + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"running\",} 1.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"scheduled\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"suspended\",} 0.0")) + // Node + .body( + containsString( + "node_state{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_executor_executedCount + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + // hazelcast_executor_isShutdown + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + // hazelcast_executor_isTerminated + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_maxPoolSize + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_poolSize + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_queueRemainingCapacity + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_queueSize + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + // hazelcast_partition_partitionCount + .body( + containsString( + "hazelcast_partition_partitionCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_partition_activePartition + .body( + containsString( + "hazelcast_partition_activePartition{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_partition_isClusterSafe + .body( + containsString( + "hazelcast_partition_isClusterSafe{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_partition_isLocalMemberSafe + .body( + containsString( + "hazelcast_partition_isLocalMemberSafe{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")); } @AfterAll diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml index e1048ab83f9..234493d29a6 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml @@ -42,4 +42,3 @@ hazelcast: hazelcast.slow.operation.detector.stacktrace.logging.enabled: true hazelcast.logging.type: log4j2 hazelcast.operation.generic.thread.count: 200 -# hazelcast.jet.enabled: true diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java index f22070d69ff..7a8bed7f982 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java @@ -229,15 +229,8 @@ private TelemetryMetricConfig parseTelemetryMetricConfig(Node metricNode) { TelemetryMetricConfig metricConfig = new TelemetryMetricConfig(); for (Node node : childElements(metricNode)) { String name = cleanNodeName(node); - if (ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key().equals(name)) { - metricConfig.setHttpPort( - getIntegerValue( - ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key(), - getTextContent(node))); - } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS - .key() - .equals(name)) { - metricConfig.setLoadDefaultExports(getBooleanValue(getTextContent(node))); + if (ServerConfigOptions.TELEMETRY_METRIC_ENABLED.key().equals(name)) { + metricConfig.setEnabled(getBooleanValue(getTextContent(node))); } else { LOGGER.warning("Unrecognized element: " + name); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index b956d4006b1..d67fc04a4d1 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -139,17 +139,11 @@ public class ServerConfigOptions { .noDefaultValue() .withDescription("The checkpoint storage instance configuration."); - public static final Option TELEMETRY_METRIC_HTTP_PORT = - Options.key("http-port") - .intType() - .defaultValue(9090) - .withDescription("The telemetry metric server's port."); - - public static final Option TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS = - Options.key("load-default-exports") + public static final Option TELEMETRY_METRIC_ENABLED = + Options.key("enabled") .booleanType() .defaultValue(true) - .withDescription("Whether to load default jvm exports."); + .withDescription("Whether open metrics export."); public static final Option TELEMETRY_METRIC = Options.key("metric") diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java index c4d10cdd0b5..a211514e353 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java @@ -21,19 +21,8 @@ import java.io.Serializable; -import static com.google.common.base.Preconditions.checkArgument; - @Data public class TelemetryMetricConfig implements Serializable { - private int httpPort = ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.defaultValue(); - private boolean loadDefaultExports = - ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.defaultValue(); - - public void setHttpPort(int httpPort) { - checkArgument( - httpPort >= 0, - "The number of http's port failed checkpoints must be a natural number."); - this.httpPort = httpPort; - } + private boolean enabled = ServerConfigOptions.TELEMETRY_METRIC_ENABLED.defaultValue(); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index 9ceb2ed68a3..3fbd1896c67 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -36,5 +36,4 @@ seatunnel: fs.defaultFS: file:///tmp/ telemetry: metric: - http-port: 9090 - load-default-exports: true + enabled: true diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java index d4137955c8b..ab27b69bc1f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java @@ -27,6 +27,8 @@ import com.hazelcast.instance.impl.Node; import com.hazelcast.internal.ascii.TextCommandService; import com.hazelcast.internal.ascii.TextCommandServiceImpl; +import io.prometheus.client.CollectorRegistry; +import lombok.Getter; import lombok.NonNull; import java.util.Map; @@ -36,10 +38,12 @@ public class NodeExtension extends DefaultNodeExtension { private final NodeExtensionCommon extCommon; + @Getter private CollectorRegistry collectorRegistry; public NodeExtension(@NonNull Node node, @NonNull SeaTunnelConfig seaTunnelConfig) { super(node); extCommon = new NodeExtensionCommon(node, new SeaTunnelServer(seaTunnelConfig)); + collectorRegistry = CollectorRegistry.defaultRegistry; } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 51e18cfea5f..10463cd63bd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstanceFactory; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstanceInitializer; import com.hazelcast.instance.impl.HazelcastInstanceFactory; import com.hazelcast.instance.impl.HazelcastInstanceImpl; @@ -27,25 +27,23 @@ import com.hazelcast.instance.impl.Node; import lombok.NonNull; -import java.io.IOException; - public class SeaTunnelServerStarter { - public static void main(String[] args) throws IOException { + public static void main(String[] args) { createHazelcastInstance(); } - public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) - throws IOException { + public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); seaTunnelConfig.getHazelcastConfig().setClusterName(clusterName); HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); - createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + initTelemetryInstance(hazelcastInstance.node); return hazelcastInstance; } public static HazelcastInstanceImpl createHazelcastInstance( @NonNull SeaTunnelConfig seaTunnelConfig) { + checkTelemetryConfig(seaTunnelConfig); return ((HazelcastInstanceProxy) HazelcastInstanceFactory.newHazelcastInstance( seaTunnelConfig.getHazelcastConfig(), @@ -55,15 +53,24 @@ public static HazelcastInstanceImpl createHazelcastInstance( .getOriginal(); } - public static HazelcastInstanceImpl createHazelcastInstance() throws IOException { + public static HazelcastInstanceImpl createHazelcastInstance() { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); - createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + initTelemetryInstance(hazelcastInstance.node); return hazelcastInstance; } - public static void createTelemetryInstance( - @NonNull Node node, final SeaTunnelConfig seaTunnelConfig) throws IOException { - ExportsInstanceFactory.newExportsInstance(node, seaTunnelConfig); + public static void initTelemetryInstance(@NonNull Node node) { + ExportsInstanceInitializer.init(node); + } + + private static void checkTelemetryConfig(SeaTunnelConfig seaTunnelConfig) { + // "hazelcast.jmx" need to set "true", for hazelcast metrics + if (seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric().isEnabled()) { + seaTunnelConfig + .getHazelcastConfig() + .getProperties() + .setProperty("hazelcast.jmx", "true"); + } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java index 0a5d8437be3..79076879339 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java @@ -24,4 +24,7 @@ public class RestConstant { public static final String SYSTEM_MONITORING_INFORMATION = "/hazelcast/rest/maps/system-monitoring-information"; + + public static final String TELEMETRY_METRICS_URL = "/hazelcast/rest/instance/metrics"; + public static final String TELEMETRY_OPEN_METRICS_URL = "/hazelcast/rest/instance/openmetrics"; } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index 1540f99c986..55b13b79532 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -44,7 +44,10 @@ import com.hazelcast.internal.util.StringUtil; import com.hazelcast.map.IMap; import com.hazelcast.spi.impl.NodeEngine; +import io.prometheus.client.exporter.common.TextFormat; +import java.io.IOException; +import java.io.StringWriter; import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; @@ -57,6 +60,8 @@ import static org.apache.seatunnel.engine.server.rest.RestConstant.RUNNING_JOBS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.RUNNING_JOB_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.SYSTEM_MONITORING_INFORMATION; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TELEMETRY_METRICS_URL; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TELEMETRY_OPEN_METRICS_URL; public class RestHttpGetCommandProcessor extends HttpCommandProcessor { @@ -91,6 +96,10 @@ public void handle(HttpGetCommand httpGetCommand) { handleJobInfoById(httpGetCommand, uri); } else if (uri.startsWith(SYSTEM_MONITORING_INFORMATION)) { getSystemMonitoringInformation(httpGetCommand); + } else if (uri.equals(TELEMETRY_METRICS_URL)) { + handleMetrics(httpGetCommand, TextFormat.CONTENT_TYPE_004); + } else if (uri.equals(TELEMETRY_OPEN_METRICS_URL)) { + handleMetrics(httpGetCommand, TextFormat.CONTENT_TYPE_OPENMETRICS_100); } else { original.handle(httpGetCommand); } @@ -207,6 +216,31 @@ private Map getJobMetrics(String jobMetrics) { return metricsMap; } + private void handleMetrics(HttpGetCommand httpGetCommand, String contentType) { + StringWriter stringWriter = new StringWriter(); + org.apache.seatunnel.engine.server.NodeExtension nodeExtension = + (org.apache.seatunnel.engine.server.NodeExtension) + textCommandService.getNode().getNodeExtension(); + try { + TextFormat.writeFormat( + contentType, + stringWriter, + nodeExtension.getCollectorRegistry().metricFamilySamples()); + this.prepareResponse(httpGetCommand, stringWriter.toString()); + } catch (IOException e) { + httpGetCommand.send400(); + } finally { + try { + if (stringWriter != null) { + stringWriter.close(); + } + } catch (IOException e) { + logger.warning("An error occurred while handling request " + httpGetCommand, e); + prepareResponse(SC_500, httpGetCommand, exceptionResponse(e)); + } + } + } + private SeaTunnelServer getSeaTunnelServer() { Map extensionServices = this.textCommandService.getNode().getNodeExtension().createExtensionServices(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java index 6547a2708b4..99a1f15e9a1 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -20,13 +20,24 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.SeaTunnelServer; +import com.google.common.collect.Lists; import com.hazelcast.cluster.impl.MemberImpl; import com.hazelcast.instance.impl.Node; +import com.hazelcast.internal.cluster.ClusterService; +import com.hazelcast.internal.jmx.ManagementService; import com.hazelcast.logging.ILogger; import io.prometheus.client.Collector; +import io.prometheus.client.GaugeMetricFamily; + +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; public abstract class AbstractCollector extends Collector { + protected static String CLUSTER = "cluster"; + protected static String ADDRESS = "address"; + protected ExportsInstance exportsInstance; public AbstractCollector(final ExportsInstance exportsInstance) { @@ -56,4 +67,55 @@ protected SeaTunnelServer getServer() { protected CoordinatorService getCoordinatorService() { return getServer().getCoordinatorService(); } + + protected ManagementService getManagementService() { + return getNode().hazelcastInstance.getManagementService(); + } + + protected ClusterService getClusterService() { + return getNode().getClusterService(); + } + + protected String localAddress() { + return getLocalMember().getInetAddress().getHostAddress() + + ":" + + getLocalMember().getPort(); + } + + protected String masterAddress() throws UnknownHostException { + return getClusterService().getMasterAddress().getInetAddress().getHostAddress() + + ":" + + getClusterService().getMasterAddress().getPort(); + } + + protected String getClusterName() { + return getNode().getConfig().getClusterName(); + } + + protected List labelValues(String... values) { + List labelValues = new ArrayList<>(); + labelValues.add(getClusterName()); + if (values != null) { + labelValues.addAll(Lists.newArrayList(values)); + } + return labelValues; + } + + protected List clusterLabelNames(String... labels) { + List labelNames = new ArrayList<>(); + labelNames.add(CLUSTER); + if (labels != null) { + labelNames.addAll(Lists.newArrayList(labels)); + } + return labelNames; + } + + protected void longMetric( + GaugeMetricFamily metricFamily, long count, List labelValues) { + metricFamily.addMetric(labelValues, count); + } + + protected void intMetric(GaugeMetricFamily metricFamily, int count, List labelValues) { + metricFamily.addMetric(labelValues, count); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java index 5f636dd6c45..66bcdb8b64b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -17,43 +17,38 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; -import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ClusterMetricExports; import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.NodeMetricExports; import com.hazelcast.instance.impl.Node; import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.hotspot.DefaultExports; - -import java.io.IOException; +import lombok.Getter; public class ExportsInstance { - private Node node; - private TelemetryMetricConfig metricConfig; + @Getter private Node node; - public ExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) throws IOException { + public ExportsInstance(Node node) { this.node = node; - this.metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); - start(); + init(); } - private void start() throws IOException { - if (metricConfig.isLoadDefaultExports()) { - DefaultExports.initialize(); - } - HTTPServer httpServer = - new HTTPServer.Builder().withPort(metricConfig.getHttpPort()).build(); + private void init() { + // initialize jvm collector + DefaultExports.initialize(); + + // register collectors CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; // Job info detail new JobMetricExports(this).register(collectorRegistry); // Thread pool status new JobThreadPoolStatusExports(this).register(collectorRegistry); - } - - public Node getNode() { - return node; + // Node metrics + new NodeMetricExports(this).register(collectorRegistry); + // Cluster metrics + new ClusterMetricExports(this).register(collectorRegistry); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java similarity index 71% rename from seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java rename to seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java index 29a85469cec..ccc3f206bdd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java @@ -17,18 +17,13 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; -import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; - import com.hazelcast.instance.impl.Node; -import java.io.IOException; - -public final class ExportsInstanceFactory { +public final class ExportsInstanceInitializer { - private ExportsInstanceFactory() {} + private ExportsInstanceInitializer() {} - public static ExportsInstance newExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) - throws IOException { - return new ExportsInstance(node, seaTunnelConfig); + public static void init(Node node) { + new ExportsInstance(node); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java new file mode 100644 index 00000000000..b004e8557fd --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.telemetry.metrics.exports; + +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; + +import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.internal.jmx.InstanceMBean; +import io.prometheus.client.GaugeMetricFamily; + +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +public class ClusterMetricExports extends AbstractCollector { + + public ClusterMetricExports(final ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + + // cluster_info + clusterInfo(mfs); + // cluster_time + clusterTime(mfs); + // instance count + nodeCount(mfs); + + return mfs; + } + + private void clusterTime(final List mfs) { + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "cluster_time", + "Cluster start time", + clusterLabelNames("hazelcastVersion")); + List labelValues = labelValues(getClusterService().getClusterVersion().toString()); + + metricFamily.addMetric(labelValues, getClusterService().getClusterTime()); + mfs.add(metricFamily); + } + + private void clusterInfo(final List mfs) { + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "cluster_info", + "Cluster info", + clusterLabelNames("hazelcastVersion", "master")); + List labelValues = null; + try { + labelValues = + labelValues( + getClusterService().getClusterVersion().toString(), masterAddress()); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + + metricFamily.addMetric(labelValues, 1); + mfs.add(metricFamily); + } + + private void nodeCount(final List mfs) { + Collection memberImpls = getClusterService().getMemberImpls(); + + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "node_count", "Cluster node total count ", clusterLabelNames()); + List labelValues = labelValues(); + + metricFamily.addMetric(labelValues, memberImpls == null ? 0 : memberImpls.size()); + mfs.add(metricFamily); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index 471a32c5aab..9f24f1ee399 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -25,7 +25,6 @@ import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; -import java.util.Collections; import java.util.List; public class JobMetricExports extends AbstractCollector { @@ -41,37 +40,29 @@ public List collect() { if (isMaster()) { CoordinatorService coordinatorService = getCoordinatorService(); JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); + GaugeMetricFamily metricFamily = new GaugeMetricFamily( "job_count", - "The job count of seatunnel cluster ", - Collections.singletonList("type")); + "All job counts of seatunnel cluster ", + clusterLabelNames("type")); + metricFamily.addMetric(labelValues("canceled"), jobCountMetrics.getCanceledJobCount()); metricFamily.addMetric( - Collections.singletonList("canceled"), jobCountMetrics.getCanceledJobCount()); - metricFamily.addMetric( - Collections.singletonList("cancelling"), - jobCountMetrics.getCancellingJobCount()); - metricFamily.addMetric( - Collections.singletonList("created"), jobCountMetrics.getCreatedJobCount()); - metricFamily.addMetric( - Collections.singletonList("failed"), jobCountMetrics.getFailedJobCount()); - metricFamily.addMetric( - Collections.singletonList("failing"), jobCountMetrics.getFailingJobCount()); - metricFamily.addMetric( - Collections.singletonList("finished"), jobCountMetrics.getFinishedJobCount()); - metricFamily.addMetric( - Collections.singletonList("reconciling"), - jobCountMetrics.getReconcilingJobCount()); + labelValues("cancelling"), jobCountMetrics.getCancellingJobCount()); + metricFamily.addMetric(labelValues("created"), jobCountMetrics.getCreatedJobCount()); + metricFamily.addMetric(labelValues("failed"), jobCountMetrics.getFailedJobCount()); + metricFamily.addMetric(labelValues("failing"), jobCountMetrics.getFailingJobCount()); + metricFamily.addMetric(labelValues("finished"), jobCountMetrics.getFinishedJobCount()); metricFamily.addMetric( - Collections.singletonList("restarting"), - jobCountMetrics.getRestartingJobCount()); + labelValues("reconciling"), jobCountMetrics.getReconcilingJobCount()); metricFamily.addMetric( - Collections.singletonList("running"), jobCountMetrics.getRunningJobCount()); + labelValues("restarting"), jobCountMetrics.getRestartingJobCount()); + metricFamily.addMetric(labelValues("running"), jobCountMetrics.getRunningJobCount()); metricFamily.addMetric( - Collections.singletonList("scheduled"), jobCountMetrics.getScheduledJobCount()); + labelValues("scheduled"), jobCountMetrics.getScheduledJobCount()); metricFamily.addMetric( - Collections.singletonList("suspended"), jobCountMetrics.getSuspendedJobCount()); + labelValues("suspended"), jobCountMetrics.getSuspendedJobCount()); mfs.add(metricFamily); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index c38c34be42d..88b3bb0e0bd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -26,7 +26,6 @@ import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; -import java.util.Collections; import java.util.List; public class JobThreadPoolStatusExports extends AbstractCollector { @@ -43,11 +42,11 @@ public List collect() { List mfs = new ArrayList(); CoordinatorService coordinatorService = getCoordinatorService(); - String address = getNode().getNodeEngine().getLocalMember().getAddress().toString(); - List labelValues = Collections.singletonList(address); + List labelValues = labelValues(localAddress()); ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); - List labelNames = Collections.singletonList("address"); + List labelNames = clusterLabelNames(ADDRESS); + GaugeMetricFamily activeCount = new GaugeMetricFamily( "job_thread_pool_activeCount", @@ -67,7 +66,7 @@ public List collect() { GaugeMetricFamily corePoolSize = new GaugeMetricFamily( "job_thread_pool_corePoolSize", - String.format(HELP, "activeCount"), + String.format(HELP, "corePoolSize"), labelNames); corePoolSize.addMetric(labelValues, threadPoolStatusMetrics.getCorePoolSize()); mfs.add(corePoolSize); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java new file mode 100644 index 00000000000..25a5bfc42dd --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics.exports; + +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; + +import com.hazelcast.internal.jmx.InstanceMBean; +import com.hazelcast.internal.jmx.PartitionServiceMBean; +import io.prometheus.client.GaugeMetricFamily; + +import java.util.ArrayList; +import java.util.List; + +public class NodeMetricExports extends AbstractCollector { + + public NodeMetricExports(final ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + // instance state + nodeState(mfs); + + InstanceMBean instanceMBean = getManagementService().getInstanceMBean(); + if (instanceMBean == null) { + return mfs; + } + + // node hazelcast executor + String address = localAddress(); + List labelNames = clusterLabelNames(ADDRESS, "type"); + GaugeMetricFamily isShutdownMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_isShutdown", + "The hazelcast executor isShutdown of seatunnel cluster node", + labelNames); + GaugeMetricFamily isTerminatedMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_isTerminated", + "The hazelcast executor isTerminated of seatunnel cluster node", + labelNames); + + GaugeMetricFamily maxPoolSizeMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_maxPoolSize", + "The hazelcast executor maxPoolSize of seatunnel cluster node", + labelNames); + + GaugeMetricFamily poolSizeMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_poolSize", + "The hazelcast executor poolSize of seatunnel cluster node", + labelNames); + + GaugeMetricFamily queueRemainingCapacityMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_queueRemainingCapacity", + "The hazelcast executor queueRemainingCapacity of seatunnel cluster ", + labelNames); + + GaugeMetricFamily queueSizeMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_queueSize", + "The hazelcast executor queueSize of seatunnel cluster node", + labelNames); + + GaugeMetricFamily executedCountMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_executedCount", + "The hazelcast executor executedCount of seatunnel cluster node", + labelNames); + + List asyncValues = labelValues(address, "async"); + List clientBlockingValues = labelValues(address, "clientBlocking"); + List clientExecutorValues = labelValues(address, "client"); + List clientQueryValues = labelValues(address, "clientQuery"); + List ioValues = labelValues(address, "io"); + List offloadableValues = labelValues(address, "offloadable"); + List scheduledValues = labelValues(address, "scheduled"); + List systemValues = labelValues(address, "system"); + + // Executor executedCount + longMetric( + executedCountMetricFamily, + instanceMBean.getAsyncExecutorMBean().getExecutedCount(), + asyncValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getClientExecutorMBean().getExecutedCount(), + clientExecutorValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().getExecutedCount(), + clientBlockingValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getClientQueryExecutorMBean().getExecutedCount(), + clientQueryValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getIoExecutorMBean().getExecutedCount(), + ioValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getOffloadableExecutorMBean().getExecutedCount(), + offloadableValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getScheduledExecutorMBean().getExecutedCount(), + scheduledValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getSystemExecutorMBean().getExecutedCount(), + systemValues); + mfs.add(executedCountMetricFamily); + + // Executor isShutdown + intMetric( + isShutdownMetricFamily, + instanceMBean.getAsyncExecutorMBean().isShutdown() ? 1 : 0, + asyncValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getClientExecutorMBean().isShutdown() ? 1 : 0, + clientExecutorValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().isShutdown() ? 1 : 0, + clientBlockingValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getClientQueryExecutorMBean().isShutdown() ? 1 : 0, + clientQueryValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getIoExecutorMBean().isShutdown() ? 1 : 0, + ioValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getOffloadableExecutorMBean().isShutdown() ? 1 : 0, + offloadableValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getScheduledExecutorMBean().isShutdown() ? 1 : 0, + scheduledValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getSystemExecutorMBean().isShutdown() ? 1 : 0, + systemValues); + mfs.add(isShutdownMetricFamily); + + // Executor isTerminated + intMetric( + isTerminatedMetricFamily, + instanceMBean.getAsyncExecutorMBean().isTerminated() ? 1 : 0, + asyncValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getClientExecutorMBean().isTerminated() ? 1 : 0, + clientExecutorValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().isTerminated() ? 1 : 0, + clientBlockingValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getClientQueryExecutorMBean().isTerminated() ? 1 : 0, + clientQueryValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getIoExecutorMBean().isTerminated() ? 1 : 0, + ioValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getOffloadableExecutorMBean().isTerminated() ? 1 : 0, + offloadableValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getScheduledExecutorMBean().isTerminated() ? 1 : 0, + scheduledValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getSystemExecutorMBean().isTerminated() ? 1 : 0, + systemValues); + mfs.add(isTerminatedMetricFamily); + + // Executor maxPoolSize + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getAsyncExecutorMBean().maxPoolSize(), + asyncValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getClientExecutorMBean().maxPoolSize(), + clientExecutorValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().maxPoolSize(), + clientBlockingValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getClientQueryExecutorMBean().maxPoolSize(), + clientQueryValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getIoExecutorMBean().maxPoolSize(), + ioValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getOffloadableExecutorMBean().maxPoolSize(), + offloadableValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getScheduledExecutorMBean().maxPoolSize(), + scheduledValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getSystemExecutorMBean().maxPoolSize(), + systemValues); + mfs.add(maxPoolSizeMetricFamily); + + // Executor poolSize + intMetric( + poolSizeMetricFamily, + instanceMBean.getAsyncExecutorMBean().poolSize(), + asyncValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getClientExecutorMBean().poolSize(), + clientExecutorValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().poolSize(), + clientBlockingValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getClientQueryExecutorMBean().poolSize(), + clientQueryValues); + intMetric(poolSizeMetricFamily, instanceMBean.getIoExecutorMBean().poolSize(), ioValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getOffloadableExecutorMBean().poolSize(), + offloadableValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getScheduledExecutorMBean().poolSize(), + scheduledValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getSystemExecutorMBean().poolSize(), + systemValues); + mfs.add(poolSizeMetricFamily); + + // Executor queueRemainingCapacity + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getAsyncExecutorMBean().queueRemainingCapacity(), + asyncValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getClientExecutorMBean().queueRemainingCapacity(), + clientExecutorValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().queueRemainingCapacity(), + clientBlockingValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getClientQueryExecutorMBean().queueRemainingCapacity(), + clientQueryValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getIoExecutorMBean().queueRemainingCapacity(), + ioValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getOffloadableExecutorMBean().queueRemainingCapacity(), + offloadableValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getScheduledExecutorMBean().queueRemainingCapacity(), + scheduledValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getSystemExecutorMBean().queueRemainingCapacity(), + systemValues); + mfs.add(queueRemainingCapacityMetricFamily); + + // Executor queueSize + intMetric( + queueSizeMetricFamily, + instanceMBean.getAsyncExecutorMBean().queueSize(), + asyncValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getClientExecutorMBean().queueSize(), + clientExecutorValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().queueSize(), + clientBlockingValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getClientQueryExecutorMBean().queueSize(), + clientQueryValues); + intMetric(queueSizeMetricFamily, instanceMBean.getIoExecutorMBean().queueSize(), ioValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getOffloadableExecutorMBean().queueSize(), + offloadableValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getScheduledExecutorMBean().queueSize(), + scheduledValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getSystemExecutorMBean().queueSize(), + systemValues); + mfs.add(queueSizeMetricFamily); + + // partition metric + partitionMetric(instanceMBean.getPartitionServiceMBean(), mfs, address); + + return mfs; + } + + private void partitionMetric( + PartitionServiceMBean partitionServiceMBean, + List mfs, + String address) { + List labelNames = clusterLabelNames(ADDRESS); + + GaugeMetricFamily partitionPartitionCount = + new GaugeMetricFamily( + "hazelcast_partition_partitionCount", + "The partitionCount of seatunnel cluster node", + labelNames); + intMetric( + partitionPartitionCount, + partitionServiceMBean.getPartitionCount(), + labelValues(address)); + mfs.add(partitionPartitionCount); + + GaugeMetricFamily partitionActivePartition = + new GaugeMetricFamily( + "hazelcast_partition_activePartition", + "The activePartition of seatunnel cluster node", + labelNames); + intMetric( + partitionActivePartition, + partitionServiceMBean.getActivePartitionCount(), + labelValues(address)); + mfs.add(partitionActivePartition); + + GaugeMetricFamily partitionIsClusterSafe = + new GaugeMetricFamily( + "hazelcast_partition_isClusterSafe", + "Whether is cluster safe of partition", + labelNames); + intMetric( + partitionIsClusterSafe, + partitionServiceMBean.isClusterSafe() ? 1 : 0, + labelValues(address)); + mfs.add(partitionIsClusterSafe); + + GaugeMetricFamily partitionIsLocalMemberSafe = + new GaugeMetricFamily( + "hazelcast_partition_isLocalMemberSafe", + "Whether is local member safe of partition", + labelNames); + intMetric( + partitionIsLocalMemberSafe, + partitionServiceMBean.isLocalMemberSafe() ? 1 : 0, + labelValues(address)); + mfs.add(partitionIsLocalMemberSafe); + } + + private void nodeState(List mfs) { + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "node_state", + "Whether is up of seatunnel node ", + clusterLabelNames(ADDRESS)); + String address = localAddress(); + List labelValues = labelValues(address); + metricFamily.addMetric(labelValues, 1); + mfs.add(metricFamily); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java index ba65ded1194..ba64588cedf 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java @@ -31,7 +31,6 @@ import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.internal.serialization.Data; -import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Collections; @@ -41,7 +40,7 @@ public class CoordinatorServiceTest { @Test - public void testMasterNodeActive() throws IOException { + public void testMasterNodeActive() { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName("CoordinatorServiceTest_testMasterNodeActive")); @@ -85,8 +84,7 @@ public void testMasterNodeActive() throws IOException { @SuppressWarnings("checkstyle:RegexpSingleline") @Test public void testClearCoordinatorService() - throws IOException, NoSuchMethodException, InvocationTargetException, - IllegalAccessException { + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { HazelcastInstanceImpl coordinatorServiceTest = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName( @@ -141,7 +139,7 @@ public void testClearCoordinatorService() @Test @Disabled("disabled because we can not know") - public void testJobRestoreWhenMasterNodeSwitch() throws InterruptedException, IOException { + public void testJobRestoreWhenMasterNodeSwitch() { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName( From 69cfb22be7bd625373ae69bec131f7a6f5a14c21 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 19:18:37 +0800 Subject: [PATCH 07/30] [Server-core][telemetry] merge dev --- release-note.md | 2 +- .../server/telemetry/metrics/exports/ClusterMetricExports.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/release-note.md b/release-note.md index 267cdf13677..2c889bb4a99 100644 --- a/release-note.md +++ b/release-note.md @@ -161,7 +161,7 @@ - [Zeta] Reduce the frequency of fetching data from imap (#4851) - [Zeta] Add OSS support for Imap storage to cluster-mode type (#4683) - [Zeta] Improve local mode startup request ports (#4660) -- [Zeta] Support exposing jobCount metrics, jobThreadPool metrics and JVM metrics by prometheus exporter (#5070) +- [Zeta] Support exposing monitoring metrics by prometheus exporter protocol (#5070) ## Docs diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java index b004e8557fd..df5cea73fa3 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import com.hazelcast.cluster.impl.MemberImpl; -import com.hazelcast.internal.jmx.InstanceMBean; import io.prometheus.client.GaugeMetricFamily; import java.net.UnknownHostException; From 9ce214d35a99087b6ef5cea5496bd4c8c4e69862 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 19:23:26 +0800 Subject: [PATCH 08/30] [Server-core][telemetry] add grafana dashboard image --- docs/en/images/cluster_monitoring.png | Bin 0 -> 557327 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/en/images/cluster_monitoring.png diff --git a/docs/en/images/cluster_monitoring.png b/docs/en/images/cluster_monitoring.png new file mode 100644 index 0000000000000000000000000000000000000000..8f534510cca1d9aa93b990fc9a76c83d5082692c GIT binary patch literal 557327 zcmeFZcTiJp*Dov{dV=gj*z_icJ06R$>03>^AfM$aOH~p73G)D-+2-3X5I*3>>BOf zS$4QXDfrsy-W&oUQYSli3-#b8s`PnlD6Vsfh|<&Z_0@K^UkSKvQ-p|AW35LoV$B8P z=>iuP=%Y*g?9N*W3Sx$N1OaL&fys=!C`SO3Z87I}F8pFQKfkNej* z?*sOY`-$WKnIMM+`z-$1uwQ>`r}}15-R>#K@cJJO`w>ClG5=_4?mZ+3{F`!+BkO;@ zbdB0K#iHayEKuLBw_2Gvs+Y_hrg5+X~w}G!}B?aUuyS0zPY{*J!*d5 zKJ~jFa1N`feIqv_Xz=y+nDT#k*l%iAox(LTMb$Yq@9SBIHPG0NNQ*nDO1#=pm3mOw zymFcPFeyoR=k$sh;8AR-JnodqYInNzaCID@e#1%BPn{{9e`en=q%#qI-=4=t)DC2;qyPnLj2KA#DsE?c>;Gq%fg5QP%z7q8L|5MlhsO6w~x^TJJ zlrv@OpOjx#x7;W6aac&lQo2=znCj1}`xbKZ>|`s5az06H!M8c2T8sRDSh?~FT*IxG zSGM4drlb8EroNmrr7;ZUbVQ>5mn)u!$Xe40C1Kk!Y58w0*;0O`c|&(=={G1|NIEia zR!RbLs!BET-QDqgFhNpC=P4W+KjkORc4ts_C#MPXIzwmU<4{&}o?$KX;g$^xEH8|T zY&#n+glgU}rMo+*gc{}S&>gGt*Qd!n*q-Z7lS&|8Uk>ETs=T%o78 z4k^>D(?DFdebsMSA5R)64*h%G=;MfGHYzQ8zTyH1nv=8xJS5u`H~hZeXD2Xex6wH$ zg-W)PydUHZa!m<1>f51OdS(<{4CIs0X$LY66_ z{4UXk3ktKcL2J~+KHcE&vQ#!P_YtLxo!wIP^%#K zxyNJzO+Og@&WP5Z4eE^ND3VFwtB)*Gt&+jv;iRzA;tcNAOW9W;Sx$N9$19Zeiq=3D z{A829aH=TaXMf6D{|xKk2a^y@xt*ZN5;*O>isMYrd1L6Ti{aL^bIX^t#NLFp5Qeh& zP!^1*mWK!K+1o^$YZ^1JeE8KSeKsruwB54XldL;tn;Ck^v|TjZ-#nF6?XoR!;Mr9OC;{$5 zO-GZlW}dURZUD!ktyj(0A7|Wh(tsSPILrG*tn=lM`5yJkU7oXBbYJLb#63u|oye(i z+Z?Y59~{Y5>g`Vz{+XvIW}cL!3uf`Zc8&HiNRX2}{w()$abvPLOKwfH+z^Z!{h|DOD%gd}oo#W!|5Cp+yk3K%`_>Eh((jv; zMnTT%MMx6upKJF^$grOC7XfZtqdJQ{3M&xx^ir;g7o*=5NF*?cuB&w`6iMifrI{Ef z9}330a#p>kq<<)F3v0>Z@>R&Zd(d+>S0xa zRd0>0g>f+#x*uKaU1lZH@)`7bzyZ7#-BCp+*uKKWHY-J@+_l$}rI7*)$g0u1oo~b4 z_zMjicMJd4f|{M)suOM(!2~q#(S=X#jtKSo*skVTHkOJlMg~<;XxtHJxiiPQ95na3 zNOSP*^ye#Fw;0*Jg6=TGKPGg?%4{@gtNUqy&ib3GgNW?!n$^Oiq4JYxScPXkjb4(D zp1#83*QTvWyFP9$s2aLd#d)&FL@pZFMQ2zUJ_tlLYPVocVNbRu zls6$OOLk>7Eoy1c-txEXo^MkR@WAtwsq{UOMl$iu%u6h z)>&rXu07+`)s4rc+Vzylbpw7SKT#NMhsM+S9n;=C9)X~+06k-1#h&PNo#a&RcF!Px zxzE~tgdZ+*fLjtU`^>e}d`62>Y{9K|KG6pz*q!mH5e3cU8ge~|Wd&|4=T)7UoN3NO zS006GQ!f#w9p=Ke@LglW0nq?60ipyHV&vEyG5Q*}skDnS7ORRKDd>LP`xVOM5vlLI zmeg5g9m?!G99KD^3-5hS!u*yi`mIu+e!c6HJe!D)kaolqD~+KCCo|v6N#IW5Y_Ik? zS^%LefA34*Xx^$DBU{JpcDbm@IAJIxWykt|FVnW1;#|VV!y<h=bbhTa6;0woB{_IxXf@&Q92O`N*s5<4-Sv!AY+M0yHmzma@7a5)x z^kIym2GCC&O3Z%6w@rM!fnpoe75*@!0jigaMWkL9gzZzLGqIu@=Zs?5*&R^4!8IL7 zg0#YrFu8pxZkbJ!?5xREo1zb1@WVxA__uF;PE8lNEBfAFJM)$ScP+vLWa6&ME7&tS z(lVD?gWgUM4Q2AFmPl!KDm;BbQT0w!>coitso7myL_-b#mZb8$&yIWjW{zuq&fDNQ z@9v;cEt_X-x9^vi&-pK`xE%3K%_{EgR)xUFWV%`W^VU`@6hQYrE`TVL#pbrL$3@)BTLYa=w>*@04d3#{ zlVqG_e~Rhaw;;6WTXC}(Z5#g~=H`6ogjbS{o}Z=97C|m(L@g{%i6Tiy*F<5**1S>- zA^rlaaG_3dHkAc6{ki8c?MFA`hhz20X#KFkIT#ZqrVr~&#s-YJ-aryJX;#0fa~MWNMgU{f{ zREoym1$a+u?*#}CW!|P4P<&$Wp`arJ7%K~A|NYuENa`WNRt%Dvy9nM1IMsJ5myw~O zh|V;cnLC|0m9>8je^)H)$k12k`ZLA1-dmN@M1xqMrnvg!j8@CVI+-^T~Dx}rLi zxyAJ2d?)4!X6s9muF=3YX7?z@DlV7~tScFPnZO^1Th)%~rA-I08@s<47HBM9Y<<2){FlEtiEl&D$+!;NV0wa2h z%95ZEz`lB$RLZ|RG2aA+y$#DSZ_LW!F+ZA_RHD@YH_DXIE<6Qk+s?9 zI6gNv}JP(LwUibZC z;d+6Qg1uE@c8u1To+vezGMx0>YHvvXczZb5sf$Kzx{D@SmDfFjZ0|z^AM2D6sK~wu z_H7W=D7NIbJ6@R3rmy!=dBStvd(7VU=U)C73qIsSzuk(^>;BArwvL3gvN{iBu3)&q zKmFd38EIe$CT|;U>dtoh(n6-&8n6URA#NJ04l4yC-e>-1l5!K$1-Yqjy0xj3sUaIyP{}3^a;1$ZVwdybvEti|zv_DVYQ{%} zmIwV|lkhJLDOR)?`BqkGbVrqUK7Z>n+tx@fD9a?00FE1!ok2~lJ z+AwBN$glDyYK?b>bb#__f-m$(9y&=_)AWF}dNjuYl%3j@%GloP& zv>icAo~Gs!YRRLyfOR#viSG!<+9*#5ePAnW8x#vrnn z?U~)=J3YZ&b9g-qpZg1nju1&@s}_wTShl40aa=cx8e^F)rbUv7c;)mgNajAUYxNOt zSA_E0oA(Wo5Q_n@iSzer9SDylVk1CrU@h6yZs4S!KKKH);25yhsVn|Uuw`C>YrlWW zRu*H@y|OXpS;1yj46V|Tx1Gx?)bL2v^kArrnJimb`9$^m1LgGIoq9E{E9x2I&zXK; zFmv8$z5+X4+2L9Y^V7D`8u5^b%+M;WY;Ze}<*t$3VCIW=$m@F^rZ zSj%I89AhQ%YU!gmCb7KuqeJf8t+!9)YK>5`8FqEkZQe&4*C|QNgkV%-f+fcJdY#h7N z>~0?k%KycRXyxw*JJd!Oq9s++qSlRU())WrvhXN+;a4W!Yd7>kCp81ln+9ih;8etf z6c~`imC?j<2d$}pqiW60P$+}M6b$iq}#NHyG;(W@y zazIq7?>7U0Gu^QwD_;iNk<4}%JIbkFblE;<<%_SC|U#X{q)xY|(b zzFOwXu`b5I%<(ucNOV_eJ+`58_NU$K;djsII*BrgfwbTi`av~uM!~JEE5^3vDcYNm zZ2v^~UentJbp7{5ra~u`4<{Y1Zq^%dp)8Qonu&wmr&J=Q`ME2&cY20a>iVux?f|s~ zoLsi_=sa!_fLU)WTOAIHo=o*kusY+|NZVv?MA@CRqAv^&SI1&@fF5Wa8(V=Hc@XW; z`9|+&U(yL}jfQL`6r&=&{fgJ)iD;K&F#EO847r!FJbT?78PxM0c3A&Um?1NMq0`}Lm<^V z12&D`>8x0U_5+;tqemJ(zT~q6d*Or2T+6bgp5?{$sD>4zIGf;@xsz zUJc3qS4P^IS+?Dq=AlS%|19J8D!HZmfTAiEncJfrM0XroFp{p}AxJxS+FcFNpu2;b zlupzi=-*SmUq0TlY=DVC3RP$*%s9{TS-eoM)A-3p_u2~C@*y4fgHJt!R>|L{_nWzB+^~ zc{nOz%_wi%kJBdFzMVCW0%iXiG64aIKalCSwTXrYPTQ0b5CynR9oEmmNIN=H3e})b zYwYH%?=eH;elNPkKn)C5+$P5vUww75;Iz?)vJ>!mJDk~C)I!?7G?`3M4 zagRDK^LC)!lGMasRXo}&4)H*KcKn_$8H*mLPfvLm9FRmSXyN98p|0G4N;$mY+Vxw% z6MQ?GByYlcf939m*x;7ypj6FAck!}0WrKI+$wJN)X9^S(g;yz6zLkj3nBew!gx=y` zvY%w;EP&8a_LU>FpVtaWwpbW*Eyl~P+hF;nkgV(zBuC4zpxE zW$|w^U26CYlP%mX@s#{^FH480PdNBp$p^(6dGv^JRF|n{y;1JBCf)8=j|;6QszGXo zIh2>~BNzNxRZaZrlU5J9P1IhdiIPh`b(do**slD^?36&pS~bU=_4>K^Zu!ctz-k2l zwL4GwDtkU0rVp|yrhM1z$h)?SG0yy`#N_>1)~d4Z*%-%1(5-hp`L$zfeTaD%7{E<` zqG0=uPhl~e6X~)eW=$`V$$gS*zH?d~T;9BR4qF``ix%FnmOz3~^)`9g{K5eofC^^m zhb=89hmcKR3GZi+=0G-Nz?((*B>@H2YOlzo$F*0V>F(_ZA7dUpct+LtZi z0%ccxD1*yDQ&!`r63X)hnaZsK!&$XSL&y3l#+3Y0alnZvy?w3nP)jfVM zrfx6WjZO7Ez1OjsNd~0gcCGsNX$*V!yIAIh!vmf%4za(QOA~)Bm`+Wh&5Kiq*qRsg zv;JLmAO(1<2?u-D&eGw6f6zOM4Rz`~GFL;MlyxecC5w~DEej$rJ19`7j_&2^DIWx=*4wL&L=SI`L=YOU_%v)Wq z?!cXYO*MYdB&GQ;trzq&!X1Jsdg6&oY%N;1PQN@DrRnT=JYK)Fl&@!neI~ zuW!nI|40UOk!Yb%=vvIkWJ@pJV+zEtJ=!X?Vb<&3tQ7LkQ1-!PiuH;bB7dk#pusnM zPN1zi?xIH%Qzr3=+%<2fx?gYuM==vJ#H0f+HFk=_kd=(NCUH{FD=4{MqHr${=HCL`G9KPHJ?nTe{Q91Fu@IoMU_KzPN&vx5@gs}IU8a1QP7T#!mhvLTp#@8 zzQ`dRNNxwa{-Y#W(EZ^YYvu2TQw-L5wI{o;a(S=y63XT4v~<07t~^;*eOzZo`tp42 z^IC)rhZ*fIzXCMbAHU-SY`A4Nyn-mPE2(K|kvmceTzO`ig0K+D9(-YYiBoi&{P=w? zmLheQv~Imln|g}wPt5+P>*U5BGk!IJGQsX{sdD`uO9>zR2l#&o7s3Oij-F0al5_=i zSC`M^(qOZ?Tv;;T=~+{kkWf&4Ip)>Nt`ULE^Q4JR+tiyEWDY%Y<@!qb+v3i`nD1{x zWAV&U4=hwF=V#|2t_{{usvku{asQ)YSv^XqoirSW1eFs2nu$>#fnmLJMkLtOkF$2p zBQq+jC5*ta5|FsZWZS#}Dw{0OTdnbU-Wm2Q~M$gT13ASJ5KFR2X_+HPS+Up$Z< ztA(@!f-;St_HA{&SS-D2KT#QeI*TZi6*hNjFP?G1yes*Bp4cEODtL{0BiuBUR6jhF z^v=1_lSg{#TYeOYF00ti>$ZkX_L2bITF7dkzS1(F`W-X*ambR~^XuKkRGN@(W&RMcq6k4=)e;_j~Y3F_> zV#XlFp>_I$YL6_Ixkp=5s>KqB@D65}^xnvRtJRWnB_k`P;wya|9i^P!yw_2ycNf^m zrLbv>!LhntLh&H0pRacQslJB?0oxjcTDZxA0^NtQ$a?k~92%t}j~;vv`{CHjLDJid zMsSEB61~Cemd03*Uti&?PR2IP=^99Z+GencPYK~ts9{13&Q<6^fMBLlqtz4ez6`=A z>j43&rS3F-BN0dOs<%hEQvATio^XfcqTp-yVer!2J;`SDyj0O9z0W!KWU1!@sOXyo zwHV$SJ3n`Pd#XC;2Hk^)0C(?T9{$!hA(~Qp4IW#ZO3WhTEiZTLyDtb3;o2!8oTntM z)FBzFw<)38^{MzysYLx0lO9TVX6cc;%@4?PVqp#Xzs(L`fBSyNDfIfR_K1DRGlw|5 zN~Hus!}C!=Ch4?>fQ}Rd@Dg7yJhRx;yV2Gj9%N>Mptlj3c@+r++XW3dm7s3(O-p4D zn!sJr_$swlO)l<59q0GV9yow!MTM(pIMiWc)`UAWiI|XP3uI2r5THk=N7$bM>`Hih ztk3Z#ce+DUu=K?$yFOOwQnYW~*W(S>@CA|-R_u*!l?@XH&~K(_E(RvWEqe@LO-G zyZv4(4rxY9wuQ?cygVIX&u*z*z+?|z2Rup#G$fgPHM9vUu9@_KUJm^_paqZ3L$Q^g zuA9EN4^QFS+R4X}RpLWJQwy0}E+lz_0<28OU`xyS@ERHuAd*8HQDf(+|IITy??M0X zCa*IceOqPgtSGzE=6(max)jIi5o`T>B#cf!BorZPS@97}neJk0@wE$n zU`qxDNQN8K$ZILdWz)Rd09)3ecEmwvY6c%Lw9IkTpHwM)Vm&jq89~65+nd=uUTd-KNOmaz$v`xG1WTOwZsxW+= z(t>h$uY9KahdUHgWx}c~43t?$dTD+*Bi1yr^;Ip&6Ku4w~hk{4AqD%63D>oLnu5e zAv_Pn9PIbZe~~5KUub3s-%wx z%wFf(Xl1!A2636iok}^X?8$N$+Uzt)QEV78yHtto%}y{(;UH;O5qhW4-Y4L&MDdW!UB2a~e`71HZ#w5=rQc za%R?w{p3f3(r98=hWF^HAK7C_bG%|@H4=u z+AI{n<9~6YRQkHoe+X2|1)mOBImkHD98hUIOk2{&ZJjrwS~z$p_x#j0*pCVf%i?GU z3{|Su2kf^Ni?24g?%_REifBvn3)f?qaA2#LI<#o|k1zUF7JirNOf{*lRn*x=cvqOL zbu_l~nxD~fQ#(ua^GE>u!jW~CzNt2^S&V6e+1sB1#X0USrhKVd`IQi@#bsVsXBIY!r?}6ykqIp9EvY=* z=8Rag7kCS?6@+}mUbPB7UK!Jr9p&&1?K@D=6xm>54Df1L)dU?Jy;WlQQ)5n`7tU39 zyb%{qDTkGJXhssZH_9(1{$jDF_QN5kgq#-H3Wj!`XJ|`jCe=jERF$~dUTspU7>+OA zy82$o6JiWatQunE3?dQWIvQG-xp*ji1GY5}0#-h=mq0eJ&%yw=f@qRWIkw2=?i_ru zt#%H8sx$k)e+oaK%4{dIS=3VH1E^knzP!;n7*BP=%_B}#@r%#v;w-ncV!pBL)YVKv zSm*@e)Fg9ZIn+-dM@S8qeeqsr=D6N1n+yRMUt@jI*@yvdN&-Ul+shKi1-}%fH*03ZQ=P@V&=^^kk>?0S)U!Gxi z1gyRX6F<1NyO67TEUMd$i8Vfb#%%U^<@`uMHc4BPiS#ZRdW#%=wxTh`!6ylOU{?%? z9&y2z3b_n)_qcQHGR>7ZT_)!GdszC0JWf72&2JWr zs2)CStkR}~{k@B7+r9T%>IQdQ%$5k7sI|GXJB7YNkM))ut?l`WMvHGRelJk3_CkOqh5{z%at zU)yHOlD3K;TqL8aI-C%5K9plSQ@s({dLe4<_Jjji#YjBEv3+S$cr<f!?(%v>YHDRCzg}7?h1&nLYp$DL4>^|M7b}CF@<@7Y8zk#>+I+|- z56m59x`R1n-HDEHt9PlbINx55po|}qz5F#FreDmMJZk)IK@;1oVH|qR)k6X2fy^t@ z@W0$0i-EbkF4*ZzxIrUXb;t#>+1?F-2{g`IlTpZA+-%437BNVA?B)&D5go44Wz(!_ zX#fSKsD-@_9Hw4;cik8q)n7=$1fQl#m9s|VQ>SfHQD*y{I)BfpiI4UXv=hPKu-ARk zp*8qknB{bZDSM5$Nvs;eOzBnmNyW%S%v9egXyxc=NNxZOuni*S9cM^(@EkG@ZOL8+kDs3*H^fgRZK z=!Wtso2c)|s-3n%GT_oMlEC6xejPfN7tlQl3xeJ#qH^a{1c_3f?Mx#Kj zAxc6AzN>QZbL)I|J-cPH03d)qdkt>)kT_jWKEPKp<(dRIBkl!Ql(?r)V&4smxf=@F z{Rdj<<8*nD%VS!hhK#^f>^Q}&;Sa7T)lJBXI}p!Vr)w7tzkh)N3@)w=HL6Y|AU@!be%wT-3W8A zX8#Z4|CAuV7D+#v;|-fF{s49B6rFzmguvu{W(iDxW~=p+Aw~1*5q02p#-6nQr;Fhq z6#hMfTW?C6gLhYH>h91N{HF(%Prd$IGvy%wULCj2XOO6$?9WBIBb4!NYNuaS{>Dsyu_%fqqD9O(8`>7tG3|dq9hVwr% zNdLE)c>Rs_&Z1TW`FFY*;(sLDhk-c+?k-<=XzKqV#Q*6hc_4wM`G2qfKPuw?mCygA zIe#7he<7bV>Vc?Kz-(;-fo$+v@dp0K+J4UMe;G3QPPRAt655>VX}nawqj`$Vn;esa z!(RORPKj6c(#fkWW_bUsgvUWw_K2+Hb8h~Z1#IJ``hoI{@}<%Fv&Mu>Lm=i(wfa@o zLRNi`Qe{C6T*91x0mXsdA@T{F>UrKr$wKbi+%JiT9@m})gS0|cD zhPe$=I9&f5+WC2}t@GX4#hXEKz{DtPex3z2Y{RACBdf34UzbSHySZrcelY&9$#IZJ z={j8bUt0Vok|3w9&8IPcFvu_MVnOU zZwZY5yBYeg{dL+uo`FwBQR+XK;{P5$VaGa^U|y;sqi_7f^Z);JgPq8>dxU69M9)-D z(rq(QXJZX)Brs*_s-$ZGn%kN$ffUycz8WRah4j~%T*sANVC0@;vto@){bT%3jWeY@ zQ;l_a|MHMeLvQg4ISF!~>Lsi%&3^%o6mn&1D*_Z1*{dwtFRm4>vH}(Z6pQk}=}x~3 zms%$BRnaaH@8W_$MJ1KVytTXGqs96WPcUkK^?&?excVmzLPyLk*fsl{R2;vGdZ3y< z38+E=uF)F2PkUyoJKomPj5AC%k~nYa099H=BrC$``c`Fr{I{p&eLKc=IlFHyz$t}mRYn63NgQI+C039hU)D%m2G z`3WJZ3^jZD&2^)6)caz_)0)3-QQmf;w7J@LO!s#rrsUNwL4*(qA{6jgRV9OGYn$0h&;{%(>s$CqN zYlroB(V1{WF^DA(Kn=K*iw1wcAft#qMM}LkY<9$-!hdX2i_DmB81qN)SJu6I_ip+k zl3sW<(0_s5bIx76#$BiO3Tk0z`$?y%sBjG$A%YgGCEH?SN4y>U=ad% zCH{uhxjazkiQi;{j-3SRtkV*+rW=o}g>&>CMGI1!ERV7t;rBWC}@?~j`2qUtfs?+`fOdbP(&|FQ>9(f5879*oS^ zbu;?xkDM|suIXig37|axEqY2BCg5=|0aWQ*?d{!bj=#Z87~#_?s>R5W`WCX8fV79@ zm;yuCq@jaNP-$7~L3kUhq{VMHdTB3xiQPquPdVYdOj7n@FGUyZ?Mehzz6{*^u}+cY zAnUxjwN8!StYMHE!s|Y?v6R1E~LPyTCWk(I+*w8( zAyv**>d-`FZJFemkz(D0TO`a*Uk3YlWvd_$wyjl04z->Y`yL_n zu;1RX&p6{)E6*!=29i-Oq-E91pMHA@<4ZgK>Wg|KbM= zqxWv7n}4`F5qwa~lP5npwj@B5KTB}Fd`me7f2mOYS9wrfw|&)##`P*X1&ds3S%I*hMWx(e zQ}?&YNvf0v8mH4BQ*RvdvNzLn!QPuI+bOwU%+y8#TNh-t-cMgSelN%h#s^K9Cp%7} zl=abL_T7}loYXF?b{q+45(7g`or33FFEg*&?0$&gW&Jk~Rnw;G{E424jFBHbrV9Uf z!9P3b2E|Y?oyj}*p+t)i=6I{279h(L~n!K3v8VuE~_Z^E!C&QP(wf0^p>`1~2UNbsmMfXc03P7ruU0B@1OJ76K_G`yN?{3k3ud| z`0X_d_in?tVh(h4^^LReLsXT?{D?0^pEt7n4N#Y@jW;N@0Oj+)b>*xXGhLQvTqf(e z;IeXmc{>%V`!YUxXENkPm(pIB+LAog^~L3L;fJPpa^1%nUUT%7J6Uk*zT^+Bx!YLg z>+EDR+S|<4c`epfWin`R?Yy5h?QUzV8SO9$OEVXNAQ^FtgtzvAF23e=f?6NmxBe^V zdUo^bd)7T%WG`kK>I54`)=s9|+a$pIjWy{`ir)Sbi=;eePcQ?a`p*BtkB= zIo7Rib)X2Ic}H zRdm8rv-C`zt9C1fG)9$}ihhv!-_$}wnNX;;JCQb2GA=d_%Y*A_!Y}G`9DAewh4t zURd5AD0?u@u7f(_?@ak?$(vKvt%ckjt+3#-vqYn0EPFYKdqL=!SomIpV9^rnw#guh zo$Qq5WAq3ucK@y)JdLz z9j3w66_;5X>o4b9R%+uPkCzX%@Q46Rf(blvBXoKXp?aAy9+nR(8xEZAJd)1e2`aes zM)kgAW}uY82F$tbi0C6$e?*BBm0r(w!Ow%Ut4iS zo6FrM_@t0NNif~6ocunUE9Fve`2=8Q$s$@VPeq2m`!A82p&mJ%u_~npIjsILwd_F| z?%*am4tLL-2?d3ro(mGBHF>92a9yjp?qg!@?kEN>NS5q7=$ywj`0p!6-#EbBWF5y_ z!pBSwF)6#Zx-)h2`_qcb?BpDr1!N9WuC3KkQG#tHhoeHj#D=b|Em3!Y@cX$J93+g@ zjR=U*hzqRrqsH6B1-sTvHDYf99Wy!N(ZRB$D(%t+!E}$7A9QLi!0{(rHreHp2NsR7 zGWs_ncFn@(2Wn09P>jw+$^_i|fy;dP7}B`l^%;RMcpO;B#254LcAK@ko8PMTYp2~9 zj4yok>UWoScAx7?*I|EYr;>bd6)|>2SS44lt;1Eo{D`Vb)xXq`8(0}BP#HuL;4$mY z(`fUe*mKVhD)|xxYWYfP{TJ-_?9NrVeSI#WV9(&Va4lr>r@OgTgSzN^(=4Bn4f3R< zUm3y(XeGbI?r0hgUUbIH58(`5Ji3*R=)YDBP-)7lblZqQr=mpwv;BVQ%b6=ft#^_K z|CH4%LybktDi%C2yJimv85*8VzWXD7o6-qq-2#ooemQ6n0xQ~ewjpw`LTfsn6;m7 z@kf&hxCZfVLB?n7gnhhXw(l3M)`ofg((H~v6}G&1*AHhYz!C$z-$CeS{iOO3^pm2K z&@$0Ks%zv-Gh<&>Vvidek{dw>`q{!p2_tPE-LL3VDS~@gt*NzZg@rPDlra%3b zN%30Y<6NGzq=a2#qKtlrqQ^W4%?AU$+B><4ga^}?K|3k`3bFLr0`xLr3`+w4^)7(j z)&*8kDi%ub4?3R>9a}YQO8a?#P*UbBX6DtAWcZEF=)2_mDGh`H@9gp(H}b>I*reD` zX!yPINln&1lSDk&5jx>BOhm&gzZ(2oJNkpKCmXrM>f_@i1u7mO2#>}_XY3mtS%-su zs$K8S?zq363ZYCO8)|&PaBa-h5VQx<35i-Vfka(@mhn!SS7Wn-KJmDEgyx`jKLNVJ2P~6&n?e!X=Vu4WA(Xyh zZKf^dD;|wKs%6q(fe71w`!7MFCMoJA97~J3fuQ&3OOTw}&gW|dkPNTn;_~x8$q3h< zi=i9G9%ILai`)Kdgnk!&mqC5H8LcMNw=Z{FT-s9vTRBDXBBugyhf>dn0EmWEw*`f@DBBI62WCsd^|U7RMuYo=@o`o4?l;qHs*8$|ZVEEQO)1z@#>rz@Kx# zDTUES&DLTa5`t^mp6BCbdas9mIM;ZG6kp0Sp|SpN=vJOTF!@02TTBewWf#~6(7#tl z6&*_zEu}-~Hj3nJ*jzrX#X}V=7yQY4OD1(|(-8+FPgGp*J;^_dDbZ>baUFLikh;tW zee$l_tLqLNh zKEV-PkI0El}3$747t>?!iBbj7)u}%m$dGM66^YMi%igdwwYQ}da;ks5;t##I-2Vr43nW%u?5=CVd~mLJeMT*O;au|S425&#{oS4I zQ_>xji=OjrBqYgnFqiP0IYOV1`$D*=4WEtDMcixvfxj^T?B;^{(Moqom9;+fg+QISgZJo)8>W9meQC8Ycw8x@}FC z04MEq&ucB!BV&9JRS&?}5{V)o{b)kt|Ha)O7c01vGV9j@>G< zcA+{|q=!MoCoe3LK~7gvY>#at+qELHZbQ_22PtNml%xm#-krt90y++K+Z=vtQ*jKw z1n!&AGr{UmzJl{XK^(ve>2Y_xg;Ll<^12Ah62pIn)vg-cvd6)j_^>o-#HPy4wwbMOZN{Q43|Ki^4)v$MCF7)zR#|K z^Aas*ZB4`RdG0SnD23%Q-Tp7(^iRgv5gqJfwR_87D{}rGV-G4vuggCWKW1?Q9N>3d2y|%f(q`GX=cNbM1&~44;r#D-WYm9j=098@Vml{;whrLQj63z?= zp~f43dn`4F@G6OD3X~`m4Llv8Pm#Mg3!2NlR?55_{jRnv29g$=XX%W*3dl1&C#~G| zjLd0VOXq!eYVe}084Ge&h|JqCGZ&u zNWXh-A?SEo9Fxuz`r))rf7U4a+i>Nv4Ynl_n5_x>cW2T!^LuTe(``F}3KGqOSROxZ zbC<5+j044LW{awEyFxJdG}|bseO7yr@e|rEGSi`ao>LTfEDJh24b5(u!r4s_ zGJPr8@P@m-Gx%~ark~011en#G?+3Sf<{UxreGXi2Mk=)RT6Na|s$dl2xc72@zM$!< z+M__?SKkyuHAY`7=miN}&A&QNIjc=#M$6`l`2-PA2wdd z_LE~!f+B*lX@;8a=m>wWc0|BJnM`mf}1 zx<)6SIGNbCC&t9KZQHhO+jcUsX2KoYPIkvda%#N)LOm8;JCafE^MCOp`logC zuVxU!zdWO4HJVBObA|uwZ`jw_|Eu58|9}4fZ$GI2^?ZUT5z8rT|LIvSl7YW4u_~!w zu&ZmcWntDsO|1=V0{SnC$O;A&WUmz9&r-|8`zYmBwu($-UU=-Xhq$3WS6h{^l74F9~G9uxCZ zDy)6(`D^>9?JTyPR%YqCT0}S4B-`WQYSS4g8{mmpRM4}PuU-9ynu0;+o@+kVd8?Y+ zB9Ny3XrApuull3AH4$eQ(BKiU2VC;p?{J?c#NzH7bSkh`t1hs(eBO1p*jLW&bqA&# zrMvW{mQk4nTsfuG1BSoDxKFmi7cZygHYCxv1~t|`13V*ji*UN(IKAtfL;@3 z=v~Ii>5bw+EU5Y|Q?(EeryH*-%$*}$6@N-5tu2-IP*~XQnZ?SylJuC>4>IgL(vRpT zw6&gpo@g#yBdNn^6X(m{QAelkPM(Vnm!?hGtr?{T*z9DsM(cjs+fJSm!M%6cI~py$ z`p0SuodGnA3eOFwHt8E#VS0-vA~~^BnS-AgH=+L5HNQ zLC3CX;up*IQJfJQ;s-_=3nd(>88VZOhsNDgm9p=^(AOq22dbRL2alg5E^_@ayZ~WV z5b#AkR{{{`!KC8`1x2 z!ZZ&*>fkT5N13Ap9Zj2w$IdJ?bl-nG8+o#G_|c+T!E7uFB21BnzUBsL6Gh56VI5yM zT|^)_sD(fK7LK=&(M;*w8R^t=Mr)elGl*~tD@5C%MD~z*9Kiet^MtvKBK%qre69V0 zp}y}p&8@FAban$|FB`n;5sIIBEYYn5cpW=VGn~NPI(fwVzf!qV8@M1?1*;!jI&aM8 zqLBXTI0n=?sb)oc=28MmT$ZtEzr~;W&mjo(r2y&9JVa_~vX9IU#$KtAxFd!cRbd&tGBFghJatV7D!aA->bM6cMti*F$i0d$+cpNJRR>k z0!PYK&f%f0{G-X{U%Pgl4zeBNUsK*At3?7Z`1PGG1tsennsOz~aG znL>>sW{lwM1J4;Y)<-L661ZS}rg*%C44S|Z@sPKZnIS0~6Td9DlI_*qNY3@Bp0C6l zn>jc+8B>64j0Z7(dxvryB?HWYKVk`m=@Ra#5fb3YpSn7Af7s zV&X9^YG}J-b{W?l`ngTY8K%9S2(> zYMVetpR9Ah9GB#UAtAYuAjP*yIX^| zrM$$eRob<@=){^hrNr!*BvUExKhSo5U@PbzgR{Q)AEj9%R8S90diFXTJWW7)yl!Vb z^wMM&*j0oeH*MIgXf0ho0V${GN@vFJbnT+k%`qoKm}5z4DC4O#8briCkc7hHi-I8H_>KwF`JL3Mku)8! ztAH=AjJgyignlrL{nDR{648%|@lY}PLtMGK2k|H;U$fw_#(S$ZHPhB^sfWB0HtW%* z8gblntE(YIh{J_t zG9z3uoRG#{X$xDfwufXy7_4LX?DZtM25X;CyGp7azMgGxit9+1=$5Dz^&bu-IZd(&j z-DoeY!%8_&3nk!HD&iQt3K`J=*4*Ma+0~g-n>R*21W2yIv&m%Q@7|%`qaBkJ#}fOQ!NyG1d^dZE-FqIS=V+Fk|^u7vT$^(`;g^$8w&5+~QRQs12E9UlEg64}RK&+>4{^z{Ay65@)IB z3TrU@xFYk`{^lam(o)*5=@s8(i=%s%F2aEE>p2TOsb8j{1U_7@G!k#mvme;M^|X>8 zJ-~D)!4CqB@WteWQ$%tp0!tOiZ4Gr6lCO1O!-%nmL*UX7F-4n;3TY%=Byifflrn5B z?b8L*^cdo7Lhk0au$7m}XuoFd*)CpIpRvoK0YTWgkz+|lVR2}-(zdZW%g84+i%)oT zNzcrGD#00$z*t515L&Z}^E2_N+@F)a<`^sLjXiP8HNRL~xpCaG{Or|@^c{rkw?5a7VRVVMlnoVmTja23}bvsG;)*=lZWh zLrXPAflUx!A3=615^i*ckTcH-#}3LeriB!igC-#p@^S=%2%)dlklKRLpiS6D3iA<5 zDb7SL_4Q%mp#KNKROw>yyiHRW|LuFyLfA@{2~0JfWv2#J_>F z34kE#RjQe`HwX#7|4o^0=ECE~YMom@R~<(CzW1UV^6ieoOf?kPnE(zU&;E#fkiZ zZ@Fdc&jVGIhqlG6lEZLuIguUmV+GsfyVfjy=5-`fkKGe{+sGM|hj%Ev&|S&jPAHEh z>z+MTg3Ep^s|0YQ3Lg?7!9Tw*!Vu^;j6o}HOtzsd1Ke>?m#`53tkTXU8|7;?u2j4F zxE$=2ew8q|So1WVW@Frjw@*7;igql|bWsIXGDc0&%dxR%6XjlH?*ht*B>m2uBpVQ|LVzF!cM1K$2Lv#MMKql#FB%Ebrqe;ohJdi zI}fwQG&>k07X1B@YDG|y!jZ7`^ zAS(l^KWE{Yquv*q4VnGg7y1KPtD_RBpPQ+&gzf($f;5!>bUb#IfBJI%U*V@Cq%AB1 z5j4EXSB|T1fTi1Z4H#BiU(yjFNqCbMa4UZ6q}0d$aO@oCsYNbU+vr=KQ6mTn>(I$eyZ7KPf}Tr!hk`r&y(S%h8DwHx>3@s68u6<-%ggkROh{@!`oWz}JaSm94+kT#MBHXA z&C74|+Sz9y!HF9BxizBFa-3v|8oc8v-cavcUg%Dwa0)T9;wqUjjL)Y|=(|&5sV~BX z6DUFTHRC(}4YI&h2TUE3>nu$r)x?t~ZDov3+J;#Z2K&dM)tEh8Gj6GXz2^wvtVj>i z5RjK!|B?$4;N_eFRw{9 zb$PeY*+irC(3TYgkgRps%VnBGRCRdTpomaV4oDm3a^VyIal?a3$Z>NgfKj8e_)KO| zZYkznZfTs5A+xsoV390sA4ZX_CObiSY_z)M4gj)%1@rXQ?ftFWj-At7PFsB#R5cIy zjrj&+A=W5(u8cQH1E-UW1pA!4H8lN&RNJAePE!%>JQ!gUQ9Z>z;yGBnX4;pg{JVPM z9M=6{xgdG9ti-4c?_;6Ba(ZNEwrg-!8M~G~dL)QM&V!9x^U_y1yV~QDtU1`O#cDG+y$VRi*b7kx0^24j1^VpuO`}Jqh;U zU={z+@gcYxpMR_!q7Kk(a&e~1pNnM$m4uwEOk%@HJ$VYCkIKZFHo=#2;3~^f(Gk$Vm|w=L7=Gch-UD>6B*ao z4#&-^S5dD;;O!mtyopr~bmB)nqh?HjX6D63Z>>*-Z?0m#vNf5_g8QqYaHJNc#x?GA2X+=C{z(zJc6?MQ##*K?!rZDgPnNFwh$ZZQ zs@x;CAljXnqZXmsPxaEj37R8+@XY?oy05^mQn4+$rLrn+h9kGewh9v_jnMtT#*B{z zT7+gH1?-mgic>iYSKqZ+d{x7E@C(?}v06cdTw>z3OJ+RQQIdu;gG#c}h&AoiWfZqg z`_ZsRi~YpH+iW9{x$30C#y2+T)+oZoB;!!K7WQ(Iw+X|Y=5LlNtF1NDGZ*+U~@ z91_Y?aoUCNu0#)-RD+Z)G$W%495C@H@IASx#ugT|wZ5tS?q^JY;(4YJ7Rz45JVkg; zB3UNGJjoB2yddXPFo%O8Pr~W+5)jH#0x_;h0nQn#m>kFW@FjV$)mOrZ-u;4Q!WJe> z(ON65Zl#d2&bHNny;V3{;hn;|L{T5sfFfH(@#aAL+_EB-nEi4 zkIhsIr6Wo8W$)B@(BqriOis1DDo4&l)Ki_5Cc8LT)zZLnLmVPMG>iM+UFJv8)t!B~Y*yq~o zJg`hJ6Q+s1Y@*ZETbXN7H|kyCFc&R>tOlM>sjY_f^53Q5Z@4->|}Zl~UhmX-HND(~N~B_OvBe$)7&^c$F) zT;6|ItGS3VuP8z}ME{eaatbvbjt`kdJ9yV=vuU)MjdwJHYw_Gd#XYyL4VqsO-;A%V zt9h|T_N*_&CSxtDvs@(ORK_^F~7fphGdRZ(uNsIU)tml!~Dt$ukIJbu^}tmkRO9R?r%zrTAVqQ z$`Q@7F|n*6eL1!!Y%1D|SM|IosU(+!Q&!sQq5OhWfSd3}uwR3Y9r9Zl>|Etgex~VT zwoyI`oP-ktRy|YJImE4zEkf0~Odn1n$U?Hjwzhx#+}_y_?v1FG%2@4Eo%@_=3FstV zgVBzge8*=g8?S+1F%BQH=#{Wisw>|IHs}=M{h;uql;B}dDJD>NOT2I8?ziY{)m?+% zusy4}eZ~yMLNYudsmhnv?Vv!w`47CYE*cr zv?6Byx&S+f*YDv$jRjlSf^IDvi+=hZ4sY7?!L6DH?~uxKw;j7f^$2h*10Xsd8Au z9O+te6=F>&S%LQ1x4#MEYihrF!9^G|rZFULhlq+7;95V1b1@uqXl1VjrxF{r8ZN|8 zaWkTUi85xnsJ%!`^y5Txq4`pcBq=O@8%i{T)%K|t5b-HCtEZ%G)1#{(asm{~4!5PT z%{&gEhTc)BRus7V4d{Oria5+(?|F-&4FP9pz*CQUGNHzE3a@Q(*8Nr7dy0#eq)HZ? z=idkPuga40(QhSkDWoN?70Iy~P>Ihu%*KTB*Z7G*o;&!7*r^S1jck0|gGGRnRofT!6!E z+5M;`|3O9-%PjrymsgDC5v+AgPyt0E z*({Wc(wcU%1)SyfmT}Li3xhJdZd4m*Qk^mTcx@UsL|hU+L?Sz(p8zr6Veq%Mmw5-P zWQiGdnk5?qJH^N>SdyU!HZ5Xv5dxdBze^M3CVk1~fV=3$0+}DM}Fk z(6ZqM!*9IgHIhH^R@tXVr8$wh$Crmc9@pTPY2{ixBC`mO2|LvH2ohFkdswj3zmrx` zZCs3QaF4i$A+6*&>PBqhZ#Nm8G>{9ZG47rD{ydAz6`DhcS++8TsCU8=Q_d+T(F_8T zQ;VjRQ;vOoSrz+pOBkV=ngFm<=6B|>PaOY)WH#VGz)g5uE`b9H$XUjk(5PZkzc@AL zoHu?O^CH+t!e1E!L;s%4O7mDp@HFz{MEZW{8BW~(E2Jy0Tji=ht_3oSlz4~E6N+Db)&-DSNHDPjIRKOT1l=EXiR?n-g7E5?iJuFavFcg zBAivLf3CM}t2&VEF@%$NFBd zcfBC`Eel83Yy-(U*3Y;G3LVM=l9GTq{Y|P19!%DOH%L+?gV>NfYRvhRP18ZTuTIW= zBZO@pG1V87r6nRl0Tq_av}z8<7L6;DVQWeSJ5QV2nm&_p?!|EJD=!S|IqC_`pqDxH z%(GC7MW8R=V>GL~b`$!rYg{FR+ehPbh*4DE(01b0YOs1MXPrD6nxEF&S3CWAy!xj1 znM;L$ib{aK!tEsqA=7*Obmf$*5atS1sad^>5NXmx>k8iPV|HG7e|7xQnh`E~#NFn# z2w1{60M2=ljz}&@t~0u}FZMGU$u~;|YKkjXz%KZ!%luIQ7Aq?V5uhL z7S%5mS{J-$K%(6xINCgCaVso7fXea#T*&qXU}E&^YKi9w+zu3q$TpO+?<=+3am0E2 zvN2U?o3=@6YlNXAVmy}lM?O&>vK9Y&%CIV;ug}Gxz`1x&5LIgi6 zXa#I^CSL*`w=d1Q%_fk&pVsqSD_qW8jlZR+lN_1GW`=r#i!g^)W|CR|hKEwRh7^Jn zyU`x{=$E;6J_LJEXM$u|`39~JOPGu;xd`9w@xmKlojvnXf31^;g1q<@XCRxYt&+AJ5znIq--N<5+nc;Mw zwpT-P9H-F9Ia=@~d`kh_ma~+E7r$3}xHIB9--PzzclPtA-Imx$#7k${7%;(NG&`_b~)YjAX_l)#>qiGq2)i`&`@D9tY(m^j*`c#Fesc#kW z*stfe3DNhtxwo0mFMwnA0O7ALy#a;#n-xtc9rL_Di91$)Uw<7x|3=iUC;GwB_sgzt z**7EQN!a9WQsNe2vLUD85#1NYF~gc#Axfhu%;H9KCH_X6*!fevdOitYszvsu9)EyF zmf$A*g}vss^@|}|Q#XIj{}1BZkl#VdnMRaJvKe7IsKhRete#NNnQ*N9yu55HaZ?=0 z?HSSdbk@SS@%lvx>(sJC&hyI&Nm@VDmO3B^)B zpCjM@m<$O%(UIzyA(v0)eWmHq8q0&V_rFs(Y*{ zmy~dOUp>IrVm8>f?f6>3=6mcIyGu~W65qc@P64qIsoX?1CIhNZEqgJ2KgsWlqI*l6pH3&B+wUjc~W=Tc^HLL)z=+_^wG6hFy3v zTrg3-2+hv>GcYfXe&Sk$HvVt%(+I&}Wx_$fFH?S<3*}=|0}FmSM}MI29_UA(mAqme zHtXTI6`B5|^>iDdUCCqti&0=#yyT}jVRSs+Tf0qIy7h+X@6f&YL8?f(X61KVnQqhb z#Kwh>gBkXy^5yR2E|fm^zI2KB1)gP3cD(i09i4(A6WojTgJ3RcIW=julN5nyLx<|6 z$Yi7kj52BPS)&XNgce*oCf4LIGKn9R)> z&-T{`vyze3MwohdZLZj)0m>AAhCX^I>@N>X9m)YQ9>r1fHXF}Xw=E9V%!pu2`xlsgG zIFG6;Esdd`^K;Y-!X&92-6^ZNrMk>CglIVN+fRk}<$ik22K{hF7rV%kk91cBPr?$z zb@2{cX|<}0e)JJm%X%AZSh?gC*cC;qPYX!HutfE=Csll`y5|P1pmKt{ahkzx>A71;ZLKE9~yKVTVXNx6t*QzD(Z%`2-{9 zn$+gYYzZL(9#xaromj9Q@*}%BCA1`f@w5uNMIw-uIgf=Q?T)dEJSP?ec&K4IsKzii)j$C6XFkF3n0!{udkN7CRu{?|4&1U;D| z!<*2i|s5P^lgkk4}gak2X-5D>DQY?KKZ!j{ML2?j0;9AR&++bW!*s%utUf3 zh4pb9f7;jJOFte3l?S|b&RO6;KNuD$7l0l;V@z`@c*(>(kC1;WjZvaSt1QDPd5X3T zDr^hvZn&6}@a-hJ!%q*ch-On_5LzixEpZ5};7#KLkMW3)0=8G}UZ(FRo)z&+BMhTg z_)^j3eYS1!wr!b@SoMdr#P29aO*X^B{#>^I3DkP$C6u$h&5X3@Q%c}1%4B?fNT0ym z$vTb&9f7*ebOpBa8vIzzQwBX8JA}HwL&jrLlh5#FBEPG@sIR5&UzhkQTx;~L*ZWs| zoM}Udrc+B==VILr2L_IJ%|7;#j0ebP@QEYsfaH|k?>evF z+boyS>W^*mjP9=;E6Gkho(}UjHJ2+uZbAafsz9 zLGW47NiLKI4@w)KzyEI^?*YwJqc-T4q9y?QdV%ru8&sp)`45AK%HZVV$=bI0#s`k_pu$pYy3zy(3c7G4U`P7d^H zUh69v5pqDQt0(Wvw@78Dy_P!G@Kw#wIMy&$!X4zrZdxjJ7>B7s2Uwo-5Re12HB? zcZ2O<(eoGK^?;1^3#S9XAj%WXe(8c1W>?gGqlfR zry*clF=bwt@!kfv!1RM=d)&T?qcfmz})oFFq++vjw**K+Tz zCpkTjkQ{>D$!=Kw)xxU(`hp3FRtdoUajT4<>q))`d(OANmd3=0BIwoJ;VXVCuR>yfeU!so+&J zriSEq8f-*&rbe?vvwS4cA0-4M~do{?v1_Z;rx%(W^_Z^8N|WIcEbHX>L6y z-nCEPWRe^Hi(`|1ADs5Q509B*B({X!$$)8%4=^IpGCzsCuV!&%Ch@W5&@(vwYMG(#$g6`e+iHhrg{Ho+=95g*+hBrLd z?y*Wq&rBSuw_&ZF{zc#{<%;4!9~K@Tz|! z$sqt4?@;&tn3y5~{?3x1OH-P9FYcyLDHve7IQ-tzxrDqox#8oh!25dtg?K(E;BbTH z*Gpl~E5F$-hrt!oOYhy3Z|B;1Pae}(TOsd+41}%z$#zd6tQV1F@}dIYh$qOzO1!B< zRBjHuR5vfOV(3ASV=vDk=*pu@4h1g)gdM9Vhg9&`{t5!IAwA%58*=;+-G_&5Wq3#F=CCfGT2d z?FUolgv~Vu2a^)@C7K!_ly|+pyYiDePBGBTNuM7CA?F&3{?kOei{m%Oi7S~qa{GwY zqfKoA^~I(#0?g^_#}P~FWgf)RQ7l(#Z?Q}*y~a%S^vNLpOTu#=9)A*HY!=xr)*;?t z0p!(0{sBL#-;=DXC)Hrohur|wSysDS3XXUw4UAm2Us80W!{Ks2YB%mS&Htod^VXZ2 z!h|I;F5I~E`^lEGe>Du9U#pvb2hKnA!N>Lxa9I!WhLONctFhiBY} z%wVfuF1ku6qH_1{+QCJ%1gAp)!wcoC!(IotKB-+<@Uyn~Z$;;L2ZZ8UOd;QT{ zP!ea<&TMiiTra-blZjYh>*b$c=$EVB%s4#rBy%*hT>$Xm>7`kpp<>xtfjeADgaB}Z zy>2=j`Nz4h$;?xNG~z$(CCtdw?ZewW|95%B;wVt(TioAK^$!qy3IV(7y(v6V7cawI zJdtCx6E$QF#uK5fu)E-=l}&tp-T?aR#OdIyb!?a3HNAW4$c^1F$YOqOsX36?SP^I& zL(gII@1D){yQ^DmR(ZF=;=rgRL%)QMu(m(4Rkl*f6KHR8;2PKHjFIQJ^&yOf*_u_K zm0tgezb7G8ud_BHM+>uvk>L0Jx=nB=^pHO-r8i-DE}^Yu;{$u?VNa)LZF|=?!pGAm z!=Y_r=Jr76Dw-)*HNm-4)nHCQa$4=dDJ7?7qfqffPMlW#%ES<#V})6=8&$2AE<*Bw z#+NR^k!0G^;cBw(Jl|ojL3i9A@mUuxnm>31t# zr{K%D!oh&7T1(&up7F`6cIk(TggbA5GO>Bf_4dPVx6r{Ho22 zACl^t04dUDb{8~(tSk|ie$N;I4xf4<3I5@#37hSbDn2pBe9MVJ+&&ZMU44^;g!CSdu6Qce<){pq+wsYO1nnf@j zB#@85rSpfJb?)RY9l88f772W(jD%q4J0bJa|-k3~zER-Z?UUDY{~s{XamE|spi zMcif%+L?e_s|SF*g5B`Bh1DXczA65LkuTPVsT|lPkmX_Gju2nN>#x6ojrJ52F^xLI zx`VEUOx^nEB3squ`hlup^k`x;Kx*ufp_5Z}Nl52G}i5+#J3M+#xwrzJ2#?|28<1Uqx) z3*3AL%FTIfJ8Hjsxgftw^>=KpDQ_br`ZFUT0bDxe$DLHh^1>a{@g{lt}#3|hu ze9h2Gxi{U|#-8uhZ5I%tFby-)Qk2141H$cF$?3{k@lzprwdO%f+`It%X%lj|M4ait z;C7bnn4|6^;+x@X_uuA24cN%xMh-}bm_zQv!3P`+et(Y#NrxO4IfW>r#Uw*-;0Y3s z+_hdGktZU~JtFwF+2j$EBemVX5z|y}Qj80aJ~zK4C|sDWfU-wE2{&DX$A=)3{F|}o z!j22PvuR#08tTacWm?(v9}dnF&;AGoN^SarH&cxV1ROx3+;tJ@g|1!f1V0x-X8>P9 z1w>o(7$!tfpBqmi)-RfL#(*h@n6vfq9>#`Y*z4o2X@8#H25!0?NIFqw6Q zbM)#o3OcTqXkfvN-nFsT3}wqyaN}NF(B@M%M)WbLqjMr&u4Z>=Rl@AD8sgI2iTR>` z)Ub1-F*8t)djKZSDOG=^6jg!F5+S2MHTO8YCg{sMgsRtojO0Q%?g#K;EWCrFdnhr8 z@5xD)%zpz_0}G$pr?!S&dT1ltPCN`8abaF2F6F|F_N@H;Ik-F`iluaOS+9lC`X4WV zLEUzHQN@RhelOciu(`0rqO8Nz=SD5(KUHY>nYfO* z7A8_T^l8 zuXet~%#^QBaa?;=tTcFHV|sgT#RBk2(G9duH;W_Ge5P$u5*8mz7muYy(g-8d<{_+5 zy%@VUtwdhHfFn@Y^$BucP~8pj&e|?;vy|hs=wiN~({+EYoH+}e?`hIR0^wGcDGsk! z3CkhL@}Xa@?J?Z)o$t@ z{@v4EIf1JdKFC%Q`$8piT5LwSreOp9whsxdi_U3l^O`4P{@H=@Xu z?O^$NcTENHd}nNjMQ-B$wZFTVtb_{5gj8X2z}#fBd2 zqlw63+~A*O0}KLqQIfQD)Pc`I6unw|+~xk`ZpD3z4zlMTSj1Rb_Izf4{eVMI8d2wl zUD<-;S^rsLgNkt`Ra<7z_{iStnq?%$1rb5!%Upi~8BZQFzCKU9T?(UEC>_EPd;$T?0Cj=KP^`;7}f*FnanCsRgg z$w<&Vfm?pBKwSsXM_c7eO})W5%1S!o{g89U-!X(znDbMFFK~Zd?2GFvg--*VK~uz5IhTh(IPA`YMtekCCZxP^7o{4qez!Y;;B zur2AQ)I)RFzwg)SwsL$v6_{POQ_=a{qA^(DWmzJ%Dzxg^mhDYp!|f}=v<-|fCQA*J9-sL-b|vl)-ixt+#@yO@^n zSG?xJF?q{Xv(=j{5#&F87e9^L{>(y_c#pKx9TxK4VI3G~+C;_?M$Nb58#j8@|I~Bp zHK}p84cbN&lmhYEpUavc4Vl^FJ}K1??DKk2eP~;9%@=t%t&W(a=o0w~@{iczl&kLV z3pm6=@9j2pNtT7B=mHInR=P&q5_|=Ytj@C=EU&p#3m(kkG?tO$;CI`+)>~2&j@E2` zQT^!JS7@1aaxiQ~l}DB4 z%qhNrlDdo>oCsfTd3wu%JYNlx0WdCJX80~u+dqr)0aTAMR?jz&+&!PY+Y=@ix+17b z87k#qg-dX(eD2G=?6J|){@=z6!Gq?B3EbmRduG(4SL(oI8 zrE3+wo4FsuQ1XPv%!juu)t2p9NkFmfS!m78N4O?HJ4xty;&>kxL#Glyg>h%~K7Hnn zn_R(BD-owV>RApj&TR}P$uJtz6};6B8fbQbYJFr+#FPG|5rxT+JzPBtz6!G#QV48{ zku_EWFjhoM=b7&pP6|MCXk=d7JyKcCQfVjU9YABdBWI9%3o5rQevusOjxG!Y_zf zx6=hc45O4-K=p58QnRA)6GKN1f;bLiUGUHbEE>h9UcNLusYkp3yAoaBg|%E60Q3Ha z_p#l3G&i6BD5tLnzPnUF>`$a>&z7~c;r5X)e(&FQhEdh&pz2KVQ>Px2;~5pgi1L4h zIf!&I4FV_Y{^0jbRe=}+V0NOVi7+|xvuv1u=vyv#49Al`j#Ir8GOte48+WR1doS5SN_T$H^9m<^Vh(Z-co^) z=CmI)Z6c*MxbvryMMTDvNQ$>wqB+;(JfQP$H3&EG6!#n|udQ}XX8}8o0gfI`vP*-8 z9hYWG<$~lwg2;K)>AORI`@P}Ln+3t3Y5YVw_I*_8>JiP-V*)w)+W|QKGKjh5`xN@4 znH8M|Y!&IDejCVx2TejVhwsbxkwf{X8b|irDu{b+mr1Xxk_NDB7h|42fFuDaI#^~_ zpn6X0W8oUUQXC$xDs*^$2|+Kpev-1d8cz)(o~$~T=nqv6&s&k0TecD11PNJhrr(1< z&d%@w(#KMNQ_1H>j(G6x=%fujI{qp_4wf-+f^8hnE$hyEMvmZKGf*9md0NTo+?>(< zRyBo#Dw}9j*QxV}s!@(h#iqksMs+S*CoLNuu(dUWthUS+5V!VBU)jSc zb%PxD7$u;+?gv^Qge4SlA5D6*p>-Q;26gV`SI9SJ{Irf7to-q zl+}9o!(S&Ax{IikNqe^nAtGsFRmSkXT!MfzlCg8wP;eVG+4F4{hc-c6c z{s|5zv)u67G3a!&HqK#9 z|1vle=G+6n`Df%i&G`?tcj_LA2At=}=X4yC5vU9_^(o~$jLzW6Z&n&Hrte(Fseq>d zo=XU~@58aGUy4*yt`GD=4!Upaix6LzA;vFHT;tt&4M64}5x!KrcX8mi=#}HW>%Me3 z&~v5{af~ypbo$$sKDTdwM+7*hmie~+ntr*Z^D(yAmi{GwE+@#kbhT)FE7mOW%R1AG zpRc_nO`$I)8B_jc+!^33V!yiOQ=q%Ro4+V1Pg7XM{#G4kYx{F@Hu+}Wtlz6TL)BQ5?c0k*N|GOw3I2>#0Q*un>g23av>!O@F()fI@U@^LLIg5 zBfW0<@OQfrvOu7X;v3QY28U?=Tf|9W5vi-g+O>;hyGx-4NJ-QjzJ(qjE=N|(tKoOk zy@S4b<|#Ky^pLRjThuFNo;6rx5XS zx}yUDJNLOP9znW_`UC}6;|2P8|H{c=O1U0&ueF673A5;^^ZY#D#0-Zamb{(k2yYj;+~1CSIrET7;tn;e|ag9<*iAzZJzNhWFdXPzRe z9F2;6M+5TpB6lTEN|HCM=U4XaPu-{t8U_RbUR(433$o1&{|mBpjsVSF{()=|(D!xD z7mJf2fwqFV8^|{2gBxU<=tfcMITuKXW*lwLIU;H6qr2i*oIxZH8={^lccCu$nFo&i z=Q?lFXg6lL$ zCZZ);qIV(^UDQ#Akx10&5=8Ia=yeQ2h)%SK9t6>QA1!(>F&KweaA-Rr%c z_5BCV4`-cy_THbijlR8ZG!r^`B{j56{oZPyxH9oRiqIO0~$Y$A8nY~)!ZWsOiR## zCG;hey6tt@pjyvG9XHYledKs&Ng}LIE<4T+7aP{IJT`1*KcZzB7YGWhs4btK?B01~ zPOHkLwQZvhAW<(M7p&wSFOgR*FFa9aLp#IBy^j3AXOk|830}epfDFIvd8p8{piGG0 zqU_c8gU`#HkV%9*TRqckLn6BR+HBF)c=d(*dHD&eSZa!3Z_#xUeB^8d4nOHojK_8} zeeR@rK$l!(vzXzucbCvc2Ng5!YBm?0cr$M;?rf(MRam^0X>jEV+SglHIi~U)N;uGWeDD)NqH0) zIil_R83*51KuC!xI0a&#-C&aiv{boTqxV6Rvn@C6)>{-gx6_V(Pfk%n^>Zw@5`PU*jNmAFCy-c62%bQR#E4t#K0+$*dSkmk>b)Lf{EWD0%&;g?UB9%g!|kk%Kv|7tZF zCclh+`+X2Sf=cxV0wOEb@L!Ck96^?*ZL#*?FwW5!&ER3i(y6fwfs|!Z)ON~0P=J9j0JS?1y{nNiifUJ*XOCY@4>r0bC^RHrnJ5dp(e^_Vn*!Qf-@g0*^+bgia>pJM)4>8{=s7Z#H**J`m(@XE<*s{;O76Tl z9N-RmIDAomw65ZKmKGhw41bAO$E>&rNp@FZenVC zBdiOL%+-fpmUCLSIdoN;^%IDqZk}+J!W~xU5&!K<8l)mbpB%@61sso;Sy@i9RD?1e zBYQ|q8N-BRJxTiQ{pu<_6162LM%s}J47z0>}17Z`zDfHw7)FN3dQ=N z;`zM9%LP1zE~+m&Q+-jrz^53sveVoggk}&94g`0Zt*@^76#hKRPyBhrFu?SueaKw!6#}&rci1Ma|pev_e&vhqD}MeN82sr|5+=#hUNV1wdY} zzgbc-g}IAqFbPkIRefkWubRht>;6dwIdSk zJT)Z3_0u-^lDXA=lFV3W53{efPZrC-33lypF-MNJ8KMqr_Lv16%RQ+6dR_dYby>%g zUhgE2X<;1*`w5)EIs4fX`-XUcRj#VqC4=B7BlZnL%s$UKp^k!4Ix(n5m3eAIV&O&k z1F$QlB-i+JrrwZ%2|2z&UvTRFz9;3w>O)M_r7qoL>p8(ke@hMv-@2Ua@(e=jDaqo$ za=uEkH9JhKBO}(b9inLvq=ChZ|GMLOsfM*{#LJvhlt&-D1=>)IysUcG&00D;dh8?B z)7(gtj*X+7svFN;b@b)O&-vBgf^-Yo`pU+P4ZltD1gF~8kW`e*VCe(32&ddfyF!a}WZ1 zB`#e_$n6&~NZmD%J$o}W*bg?lc<`n&BTef%`8whtAQV0(hgY)VxU88N$lVr1%0aLB zYx8p(>u5mCATksc`pB)~I=|J0%VO1e(smF@FYXk;4_>G9%5O!H`p-n;Qoix&E3Lk< z3h)kqn%|y8e?}jWy>IPRUEO?|8SGr6_Gr11Wa$I)_%84OJwjX!M^de)_k~hF)fdBo zj!YgdQdlM_*_uxrpos13Y*(W*NqZkCG*zP@h8RPadLO>LcH%qH2_SpkFBOk8hnfCn zV2{->{FECbfoTMPj2iN`2}XATREc3_q*Ki9W1qR!OCN}si(10MU%>s%Z2bdIY54UV z35rqz2VVRkbew z>PR@lqvp4q(_H0QVPecpj1E$sz|Y@GpSaC*N%O&D%=DT%^u!45=t$Cr*CrO*G{ z8d8|zJ30|35)+Ukij_FjOBaoaAoSOFU|y`P#z%vrMr@=WfsB46bRYJ>9233l}2IBxKY~ z4=J6i!}1S>G%Q79hVsL)*QqWB;E!b|N-Q()@;&w<~sZEv;adKjXs~FOW{Z~ zrz*lUR2IPX%=_3DP1arhf+|eAbV_kJ>8ZVHi(dR9- z<1g85#5sZQ^{*$}{w6YIGm(@%>`wbM_}F3m6qHEmD5%3y&VFXqonoR?SleBZ%(+MI zV-5fB$Z0EJVraG7vx=|H+b(glu2wYy1ro39>HZ{p0=JFzx({jCF!EQV=P}Sls7VIU zoc!JpJH9!pAc=JP9wxi20wg&=cDng7fT9szONg5d@7mlrbip5Ef?7t$5p(aT*QVR> zWv1qal-~C8iviET#Df$c!%RsMh)v#oz$hEo>5WsrT8npW|BaegjsZMXN}^o3Z|KUu zh3>lv;oGB9dGfFEzs01cKIev?#2$VAxIET?7G3J(1oenb=FS_2$L?~r>t2L0 zE#!z_ z-PFk&;9*QLOkLtE@wZeg#`Ic0=0C%#joy`Uanm zqU{Fdso2I>h@aNNe-tvm{g#V=ywq?|By~fTl}GXQUa2VBHh>S@&1Uu5W~r|HQbs>L zp&`$Doa!=94LV%bC|>#S<0ag|TDgr~SsqTGA_~Mi7{^A17sFOJi@_8%t5)yC`L4^j4)640x{Yp2FA*RmxwB zw|&r9ZzA=U!F8TrRy0k?zcF$o9N-(woWN{)+Wd?)>b~q zvhB$fFV!c@E?F?%YtoB(g7MI=qFQ4{{lqj*aecAv=Us9t%)Vr^N1a;U^tW>)i+PU& z7Edbq1%?u$|E8c{x(g4I?sW!b=NXR-BIk+*UqFNaE1Tz@u*b?;K846a^%Y$(RfGTe zqBnV#@sgkOQUP&`LM-(IcBg_|^H{o$j4?P`C4hOk94Mds;pk6H*g<#N1?B+v=gl(M zVeQWaBMg~V-USZBsd{Ichumw+n#Ljz+Z}Mr12E1qF^RWTV{hBrJD&4qKDKJ>u);o? z71<|b@~MoLPm1HJ+y6-V*y9N7H36)dO)dRXnMjV}Jm0Yb=@XwZ)IXCGRnK7>E_oDo zD6h{zp28Ra_NU2|G>$#N23o_{ zC2^Qa6?+L!mFf)TYbdwVunkMP5!{xB3G}_(u-w|;`>g&Qc9|{lB4!jNvG`+@aV_WZ zQIZ<^2Z^ulCTv5BIH8;lw(=Zfa7Jf;xPl+w7%OJF=_)n%x{oIUOgV;MTglJ!si`GY zF?#QbM9~&IsD27+XFK@ep$Nqk>!GuD_X{cGJq9-fx9JuAfavh~4U*~|xTb<8yzwxn zXQ1D9xrMPJNxX#R#>7XYS~2XAn9CkV8RIF*BRelWFb!3ImGgSytFaudz2fBWKT3w@xj* z3wk59SUe0Xklj+mUeYJxKX-V_v%Jn{TlJufqdkVMN6$$C4{c)aR2u>v77$OzZcgIM znOqODxrw!eZ#fRz^ui_=!oNO%e~#V1s$rX^Gm^ln+L4z&I}$4SR5w|o@8ENQd9XSG zGp91Xz$UHI$skS*iM@fI-GU<}jH*kRSH(x`Ug5vpLUGen-R$x?e!R^D1)s-AAw(gm zPFb-T?GGCM&nWjb2pVzmmUBV-pDrYLC}k2zF9c+1BrK{I=E&7T)Ms4s-iL+Y6Lc>3 z;Ks7IMz7Qhph{(=ZA^lnJKlI^-`Ln2?${N?m|6^0?LZqScplZs&h={kJ%`6O_Ko!P zROgI*sD75IjTNwIePN&G(#e)Iii214#F4AwX|96S_CfEJL0FhJL`>tiTKs2yf9t=H zFmBEP`rWAjI5krEvnoSX^xe3quM&EM4Xq4$ED45S*sy{903M|JhzET2vGOtew=Nu+r{D@fG#87i!{T743+TnP;GiY;QnE2te& z8KoRaIYSk%{vG_*lxM|>n`Dc+Hve#~2SB+*g`PYZ0C{j}r4XE7K_;nyTt+@@@+jot zTc++Oj;1|TEH-L|-PPv${J_+e=ajn~AAUImafI;1EEO|wa-}!bW$A7S}Q&kh9OS1XlomXR5e8c7w+_zxGpf#7b z92bt8u{3+X(0EP849qHb;=!@r(nXC_%Zah8v95N(W}QFw<2cvc``F4%$R2su+7+9&N4I*D zdBoWtYNNzAW8sG!h2xr1I`QnH95{-!N=fOv->xlanhm{Q40{;zQNYh}#_Hax7FUaL zsiqMRXgUI0Zv~=CSPYFO+WpFE41|zn5(rzIiaOMcSRyt6&|~^{-w^qcq4`27R|Osm zrJ6ZV3=+v?E1Ex1dR^x+pxg3NXrDGgsK7pTx~|0E2=(BW?BJe$Mr9x1cfcR0$)DA_ zBwfK^KMgWiD0ZZvLn=pHk=*sYiVb5PeN4mkzW>siYQ1ts(%evVIVb#M&^sWtzziniBK z?rX|z63@#hygb2&t@EpgIQN9t_tJUQ!gPc_E>}$w;DCOSd9!2PxcKlH)Q`wuzbuj8 zwfU%ScWI2|1D7ohBltm701*JT?n;-&-;Ey|cNzVW_Sfh-t@GP8kg;c;%=Ro*e@y3kM-hj$^x&At>jPG!Y{X1 zpq7iO0Xrvg_y^`ZurSo2({=25rnZIWVN~1?X_Yv1eCQR}A(a@pWSdq#PcYT8M^70q z1fDG02^N@PH=t(XXB|Ys@wZ2T4$s<96b=b%=N=h3A3Nq-qSGcf^|?H+F99sLVF@>q zZ1pD9X}1`U|>`B zntQH}LJnE!PlqYkxwrRBmsDzZ8f2WT)Kb!M?UiEoNocwY`a^dBExT+1wFS=u(x)8K z{A;@TruGT_ZB}LO=^&<8_Zb*E>=aUMBRiL8Jqi;Vb>igJ`o0Jfj)Z$S}sM|35}o0cYkY=>Ji2 zJ4%|cZO=o!oh);b(mu>SrJE(D8gP(lyVs}&uziKNXyK-jduOYQNn>GxaqG%gMY6(w zzq#SwxtO0;x3k%;@BaHZaA$dJy0p%V+@Fc$C}(n`-1A|68#BkN$!=!9LfP;f+zBcT zEf!9viRnaK$K+`CIYgzBlXk}_ec0Qz7ixf&>Laz??Ib)Ghm_xkqY$dRA7_*hRH{TRt}Xirv2DK6s0I0%cV2SJAGGV6GF7!o2>lp)3Uvnkv7=xf?)? zURU33Aj;39L1(9cyL%(uUir24-K8Wjh51WRfcHF#yW(7ZvcutStr?T);#1rcEn2EMbK>e_)u zK8mtXl~Wr5&a>dM5&{4bJv_M2#IX5XLw9s)aHm@tJc{S;!eCATFZB)SJ3S5%_EYJO z)dKeqwLdX)->{_3%7C~Zq$AVBy3`oS3wOfvuq^~mw<^ljJUJs$fW+w^bycivOn6(A z`R&%!uh1Hb2R9rb#xQbH4qF8;oy>kOX0?^=y%v=oA}?NJJ>(E>?0c%~ygw)5jb0g- zwV(?jaKP!=^+i4C9C2F#1))Kg2e)1IZI^e~%uH&tUt--YdW0YfSaGJr^E;Ch@F(P^ zk5Uuik#R#e>llOmf{ier%q(UEa^T+jSbi8-uDAw=!C9trB#(Ia!?@&z= zHJAoyju~TtmV52lm8O5}Jlk$}*6{i@1A}Ei6-_{$+wR%s#$79w*;b*rMdSp2#KKil z+g_IU6>-1@((h^uc_)vAT~LB3d+Y17Zt?F5dqXjrX)(!jO9*f3(b!WT@=uu7I2zK6 zoS25)xrBp$feX)1RHhba>Jbi|oSMqXE2>>Lf4#IUE2ts!h()wcMSa7w#FHC^McF#j zmpHPUurDY_h_3`gvlIY z`Khs@zF{ol|LiiWj>rEx{=;b1Gu>7J_9s|DILZVOMx5zup7_oX!r9+HHLh=;Rn}l_P zvuzggpJx)fnJ#(JF-qDNWSq;AvEtXxpVPYppF~Su=UxVeJ@^q-P?|KBZV@}V_yQ>M zd^8FrR)i((&O9p(8fSnC`S*0_AWPVE&ArYA%(1cTZ)OgRd(?j^iMDPS{1{Cc`Vy-? zE63;W4XfR7S)N52OFa57= zml|S;>5EItmw%)05&yB_Y7zIUZ2T_-0wX1|L15F9Eo^+h!FEzQEOg%kdaZtz6qzO1 zqR$-#{{SJbHa0OGmN&Tlyfpr95F~-Ms$!&R{4ddn8u6<9-Y}5Zr5yZdBmR1fFTL-4 zawO(RErSpEhxD>^ID6}EB^E-M>3G1v$XlcMZGPaMVkr0#|@S77+DKMx6l=+;Q-?}v@k-lg8y>dZ|R{r-cWCE z&nIEBW90V_rK=y74jmUy{5LXfcPv}bU(7TYv5 zIRZzEduAMTopp*cKYt3=XEu4^z3!NGQ{wJ?=?v>p^K^!6QFqL6dI0ex1G;p4s*XQ; zO{X5AUWfq_TuNV5>cMHPB_~?WGk50Y48yp?dJhX!+%=N`a>mnPs)!cOrptr*Oi1Y| zI>Tae>GnGp$=B1potl%KF0)=%8Gmza@T;6*QuvnRE!R~K2i!K)Z({-5ktj zex=YZ-Sj?8mVp)~6YmjAzhXW#z7h6wFvrP2Cp8GJW3x`s{Jw=~FlyzvE@(i_i6;XJ zjT_>~48Vmnh#PbYHd}`ck^EgQ!jAAeWW&<q-EB+wK5&IrX3u zzuSv?<&Pk+tsfCz;B0AVD*EZ~(k6_;)DeJM4YC5%%=AlA=hS_A21C1n2 zNwOjrZHbTR+yYWj?IQGgpOs&Ft|{siTQlIqUeEXmyODX*SXCQG5QpYK-o{7{=^5?x z24__Q_%Zxto7fgBvcM&sPsGm&MQc>$2mG(n`-+>cl?Cg`f_x;;#Lu2vqYwbOwhsSq zbV`f^3{5jN(LZ5?Vvf0Ng&BCe4a za$?x`b*AWGX0vK{&45fZcUO4$TR%sYl4~MHdnB6Iad<{vZkD5n84z|#?BKLwfrGkO zO7>i=F+-xG{S+TyWSTwe;@$$hvMmb*g~Q+9AAyhi9hxZCu;t6JcQ~u~@E%^Yb|diD zk`|*)O>+=?>Xk)OC7ZD8aQzqZSgQcdnAueis3oI^WYI%V-a{{PHt0p0NMP{@QMD5} zoNdLwoHh4GBTJTm#L%-kIVMP#(`p9hZyPNE%7KYw^v9@r7?}Me_`dyl1gs%_uTBDX zjs5Q%;>iJN^5pd)5U$vh6DnjBS?6mH1==+#_lc9WK*{3Wn-!c-U zt&QK9Mu`8?5S|u2cIKj`FU5Ov7!p`u1gHEU}HT0v34Elh~wH6Pv zW&72XHpP0+HLJ--xXigT(^Sud$GBL8h#&oRc_vDNTG6D~dhOv6_;B;maa>V$>-@P) zN8yJpu@s@v?;N9FMh&Py9ZGIZ)ws?ZnZ5dbxCIb_S5Z~Y9ypM9Y@a8RJnWVEV&^pG zWGxDU`90MY0K~j`ITxze#3@ghE(CMS{j}Ui6O6##Y{#g@+ueg33^-uve&Pg!PRMtR z`J8vI?yspk4&gf3`TO^NTjs|X7wS%Q1@!Yznk4X)ogN%iRvC+`={BxBs!;G zvX^bACz?%P`WWY_GPy_`q9Wavv!`MV(jEW!g^B8jep!r8j*`3q>@@WUT4o!77gH5% zJjoe?ISK7WhZz!c92J1SD>sb{Cu5xbVL>Cq?hL+Niu5LhvUFhx#fPU>zw?K-ay}y~ z!b$o!iSdQS1Fj~f6_QJu3(Pk?&v#hkCAx>1Hck_x-hq?}seCG){v7Z^a9hd?aoDM5 z%;lQj6rU$EUVI{n1Y@?P$wp6LiXV$kgTZny^Ij)WZ;#+#f7Pwga)Kb*uYW8Yk$rk1 zQ5hnec}6UBG_naCXtp20>En7{+#vKJ_6&hgyWI!foYLQ|wcVU*we2wbUv-eSA>==~ zc9!@&|I95`e)K4IP0NMX*qaX-@9XzntdLxPwQ{|+XP9-EX%E6M)n|MhDZS1*=x9_> zMy&Vp5r6D&ax6iaESINF;ZHUClX(y5q+J|f(bSYt_4H2*u-JwVnlVYiL&Kl?V&VAD zPVM9s$%AtkOE7;J4mEJz8*;KCF;NFBsg<#xq%$|{8(T$PGE zCyX+D;ND}GnZ!YJqkxHaNegFZjzBAIfuk}@{W3@X`1}yEGXlkin6eW_E{pWVVr}6) zxH$BurlB*T2}?<;!)AC#J0oy3rE-5-h&R@G?9~d!cVTbk{yJ;Sl<+7uu!}vrH=ao- z^Kg9}BIPX%a>)WmaB*CpD@5odGplsqGhX#G->xCDt{3h)5zH4?w?*|?7au^}*X1Aq zzN>@WRUJL(+-jrIiF4%5VbdoLktxH;OSn6oz<GhX!XMS+Q#3Z_?KEF^5wRoe0M92<_q$MunX=j?UZDqgSQD>}fbc(Hg|eQ>0`)#>V8Pn~#VO(_32NyBB-fyHt&fm!go z@jC+Jb+%#s47C7uiYuEe@|0UiBJj6zYzYV#@bv)s$#vmw{X(1=Oc8yca-+CdgOk)TO^Z0 z+dhq^R;=BPYbaE}7vw?zgig0!exN$z&fADzal~-kpkL7>qL$+uG@VB(-`v~Dm4xPi zZ?+W37XwKQOnI~Y-D7XLdiHH)Kn}uvHL5v{W3!^qbz0^AN|9BlIoQKC#od-=ze^Pa z(E_kInMx?$6ts%Wj94y9ozupCc;%1_ZmacVCb%)(^f^yf0C{|XlH3`$ z@;O(<#l5XJY~vo47{sC!;>ADQ!_TH+Hgm*;8r=$?o}Q}RN(Ef+W!+{3p!ZPFje{fr z0(RBWK=e)aC_Dc5r4k|?l1W;&R(R_=(!w+~d?PhnrG-P`;;OpUK!qjz9??mel`3aW z*7aR^xN2M&lN0;`17RMutal}({n&@~QpujF3+H+k2D7BdwcYi4) zE}VxzU;+}z^XV6V<^qct0k4mjT>UC?qkrtyll{n!>t1z1?PlC|jz7w{?8j#iDl&opp>MI-c3vjBZ@b&nN3>=A7!yx$4~w z?2+>eY}(->yCc~u8yY9uDbH{+$2%tAbd93kM&KZmS%YWy zU;Q;0x&m*z8Ol2MLPBqtFH#nvE7k#5Pfk({G^{0cOaSU-mC^LA724}Y`{qYy3F1yoMH`%iY3pRhat)SN6;;ZTFvx}evzU*(Ssv_1t% zVSassJMVR}^-OIP7MomDLEQ}tg^3IzWWCqW9rq$G6{2ngD{cLPUA-E2&0uz880u$X z?%~Y(;hQ{OnPVeUWg$aWZ*pz~Kf#xkkj(Whf^tAw7C()cpH zlj_mGKkc!dxg5I}zMgc3jgU8_2FN%3cr$%9gz*6S8A4CIGiw}oSd)84T$tQ<# z_G>&Qum?Jss+n+p4~)1XIl=3bMl46Y@^5%@N7w1AqkxwC#2x?<(IB%Sv%gl2E7V;3 zh`{8!EG#J|@-(Sau=k%VfImgYVuEUCyX&{@d3EN!T5a`pt-6^6+|5S1LPwrE`mleM zocRBgT(8(X&p##S_ouzGqeF#okHVYW!ULnxyZ=OF{<^>JvHdw|8q;AG7?R5;=}<@0 zkp{S2Q2lJ45OO;|L^cw3WXk22=}a?ZnOR$H+8@mens?_By)tET8w4D%U1jg=EQ)<; z7FCImwO}f=M@8SrAL0inh1vbsuIZ;}9S<2e?g|x=HxXVDTchcU6&JYecdIX^Myf7* z=LR;=QxpSVoPM-D|GiyzeoA$5I7|#Y6at^yk;jP7#Xy= zE~c{6El*UcOaB66P+@o@>BL0(a>xZPY;_{A=kOXwxZhcr>kmV;4}{E?;oETVb%)@p z6;1g|Vs99$N)`)1U|_W{AklXJP@`Dv5Yv!xE5w|Wyx?WXERiMD);!|we-ALKr?36p z|B&eerXOE+yw^tpZi>t;@2#;@!q>B46ygSXF%Ng!5&ssx`H*S3h(dKw(71M_j}0Dx zs0YZ4t62jNdp57{nOQd_`1ypNQP!mm+jz>6QX>+xac?G~SBZg_R*Kcu%?8i*uU%2(3q=1UJ1Tzdun zO48AC3xA2T{`NZqxA8`&OQnO~x*qWu1$AKcupXUbDpd`)F#;RX~@30t; zGO|P0*X7{f^&gA~ovD;b>bDHv-})?^k<+bM3V^kX7_Z0Ftv{BoyXyJW2KnN+Jm+|R zx*a)%vgzQTU#5&2@|YbZj5?S?c{XM~NwNWHzfxA}kjM(+!$!!6WBEw@bOKtnF z1N}tS+3XLoo9j>b3L7g#Hj`Ih)>oYB5$Ro{PVkSzxY)TOExT~Fy|R{nl@cxOe*}xG zh4bnElu~aRw!_@v_?m;I3;9d^r)kD>vg+>kA=SJ_uqL zZC#d#E<2Qcw(*8N%GS~+?=9NM_@nk|i~ox}R4V(X(!KT zIGVLO?IYc)CJ)^23-h?`mO+|+C9W~{9|9Bi1fPZ7sQi9&8tv-ugX=HGT*H#IHV*h1 zTKOC}%JM+0VYqFz*XI>Z@Wf1Pn4m}I+URBVUov6f2Ca+8XuDJAar8I}93kp#HE^s3 zA*a4G*M|p<$Scd8#=ja#P!vV}6`<8D-8esDrE@FPUQFv=NiaH=%14JNx#LZGmL2i0Ep(RK4o`+Ld8I#q$Wa4$W2Oq}4dKU+ym|*>N&_}1M$hH?hf=*llcwEj=}jUBa;LsuKjHm8yHCPM zZFv7w-UQIFMrib%y!_>?2+U6G1j^XBC`PJQ8w-S!&NyRtNh!ARK5E!lemfQwqltoK zJeECEc`sZsXB%rrwbqBhNbQ=mmER{F4&;KrdS;YA|3Naj418%^682TriLt(?nAxAr z7?|U;!0dpy{dkI*eXQoA+tedAPZ2Wzo@|doUSenjZxCX>xOg!y6g~DOg_xf7CS4b^#(?O>);)0i+k6 zS$mC*=*7@}-@ZR$vxS_Z>{oTlzjdu)_+V`({4|aAUzxj;!XhX1dx>_%A1uPZS&&Qx zq#h8S2AUGEVwr_Goc)pl@}{F1nY>`t0&rr^ethJ@)H7HQ@$KG#&QbKkNlMv}Gbd3^ zS}O-Y%WTW8Ys>t>Pd!z?+^D;trsy-#GlmuLl^*ye0$^aCh-puTr=x0pJpI`zuG}qq z{Ap|8&or#WG)~8^XRaF?8|v#d=X6mXd%>8bh%jyQ@8#w)Cm(f#wVp;eHs>*gO@)9h z6DBHSLRyy>;-Ty1m}Ja2DLi@VGS25#!WaOnf2>$}2F39}D>g2O(@jw+(+tT8AHTD? z*GCocx!$noX^|>RffIgyd6%+71a*xLKp7UNXCWnw>j7`Oy>{j-dTVR% z>Nue0_ly{Rn!gS1w{Fjbo5?3;^~-Uw8@!C70kic^2jvoeF`1Y?AKEDE{G10U0#oq< z-5U3c)Bqr1GBhgjK(n5C$2VwHa{oSoU?{UOySGI>%xptXKWKR9Lfzdr5D4-$cQ+F( z*qd?rt1?_a%<`^z8F+z~mf`(6dp^7TH$rsFCO7vfCGFe)Bv5qiTJfDx*VE~i-L|y9 zR72^Pn`tVY*^n`L>{+etzZ^ZhSAYckhr&BS{9MIaXX_EZDWwXbzozXKnjs1{v&Cc0 zfQ(Ft)#d&XKR9{&DJv_&zNnhWB=J`&^M?vf$4AV5TV&H#A{Fy{=4B8M;~5NPw%TCX z_bL2XK|I$is=2_}J&b1As~ubXHobbA81qu*#z&Y7-zGM;shU@xbodKLPhQDB zH%(!*ml_&;tY$Mkvv2r$WtZ*Afy=+tRc?ZDY5`e7jcI@X>l%jg&@OEd#wx)_{*!7x zh3C<_+#zv$?C4u-n%kgB0;L}&bsfvYm-v&Dk&mPZVic%del71L1`epOPP^gUvuBv& zb(mE&Z*@+T-i{kdBB77j^|vE=^rY1-FZQZfx)?c4C|#$PBijAN?=|-pAC5xu>iZWl z!EzCdicSM=`3!yJE)!?L;LE(xSY&{+6$XpafIH>(fB9>Cin@foA4O;1W}h+{P|s49^JGpoc$dR z*Na`VSN5Ztym85oWH{Tr@*KL(9soXHH`K+;o__KF>3D_LFL*m2FbF>hI<3i-bkzNt z2%@w}3C};b${>~Hw(NPdz^0dM#l5>=k&2gAwvlKT`HYjV{k#;b8Fgj3&R7;&T^bu= z5k>lSa$O$raHH|%knuFvM&dzUdUdvzC3O>zO8zpnj$H|rPq$0%!zF&s8nScGj)BAxFS-d|lIM=-iK6#?9t!FYAxyL*^~dqQ-kePH%*rt41da6qrIg}SzYqe zIe48UIBdH}^Ch-Q)+wo#I_kO8iu!M$4;m)B9Jn zI@c;R1GKIG8~na!dV4kt=wi}_{_$Dy(~JTBRMH+L`k{_R>^aW&oCpGfjS-KkJ|WLX z0K*s2ZnbNYD%VuvB(3lA9v~Y1E$;iBBhyZ^dZz#l*27h>fAc0JZeS45adzs3EHrm; z%44Atb%PDk7uaJWUI%t9B%|xQRb0?cw^P;Ju3Dx%9}e|N)AsY-T($vMS&$vX)gkFY zV$MkRWvFxnL#O<87he(yD8ur02?3sW9YV`>>Yz)Bt6`?u=|W7tvVk39$UuYyd3$Bs zatVOGBK6d0QJV{?;I}$747lyR-N~km?ejd3d!elVWrfs3$w%)uI;kM*&i-x(K?+@Y ziG-f-KqslC((%m1aMu_!2fQqhz*ImUBJ<*g6Q2$B6zeezG`7vTvBOxBsSB(dP1x)iU_4=BF1k68#CLf zHEwXDLV9hDnP=R}AX}Bs-ow=EK42W~gJ#ZbC$TGVDqlki?nI|9T7CAtquM)ES`PC8 zUt^NbObaO=up}yFPeW8B7s)8qS{<0cQD|1q0DWyzWp3sjoSRW3pQ`HE6|wPDDn36w zQI~ob1ATqSN2(R>yx!E%^vPt2+@Szz_>DYEt@u*N>mbX?Y9&<~@8!}${Uk0i>Y{HF z*OhH?{0-O)^_H2y+`y&r;si@8XbWo-uCKqA`$Sxlnk3UVt<3ePCz<$&Idh|zI}ve~ zmx>Hj{$i^>)raHbLBU7iDPqm9S@BaLbOEvFueA8yOk!o%;F%3h5*Ut9h5G#zQrnY` z*h*D$)V(%=n1H}vouECU`v6I1r}Jsd=w0XC*=fq{_TBaA6%+$HF7{^r< zX4&NRSU^L&di0Fdr)kC(GFaxLP$MQPwt=fRM-FNEBM;u+gB%1D9UEgO!zcPaR=~-o z)PBp3+We;^JUt6ii1RTNT^VI-7N~O;8|!%aiTLB0_VTItgWJ%}e#yP~xC!N$rF)f9 za|sPlzNZH>axr=jaJGx~_G^x8OFJ->ipKrfJqM7i4UwMKUav_vr4oWSIr-z6B=xza zUC!P}drKA&Q$|oPOhY37i(mhT)xA{}y4M=sW-tZO-~Gca1rRyAeU=2M_49s~Zp@wf zh0~TZJRHqle6~o)>|3)?vCyHAIP1;wZ1rPPy9(WJ7vDT~xAeu{%1DQ`Hqke~^ydt6 zZ~-}ASHWpl%N#eQ@cQ9e)BlgHvkHnUYP)oBf+rB%-QBH`V8H_fcXy}p4i-pocbWtX z?kicS@YVJpUw&m{y?tnKHa{HQy6ISeF>7Cg#Ci zgCjd+vLk6RE!g6xCkXx3VK0+^RWN_mANfJO2%sOquraOn5W$df5QlI~UCLry{fGl~ zJsGRvzoP_0m>%gmZl*W88u!emEP{`tV-K|N@j>#P{wyD8FR;esRL8MEHH8;0DBDG3 zyt8|P4?y%$=5bJS^bkrI`1R=ZX=&}5;`OQ#0yBPPaE6=%gUjne5{QU;hVOcYaBAp7UG^QZ@ z+b*XNM7S!%TY<~38g;!rCbsZq%_E|*8A~ogukK6gA3J%PxnNZGiFOn1i!Qp?Wz3BG zGI8@&C~Vrc!?k4B?J>=HY^#H*Bs>rPTomkCcqLq;FxEu(f$eA@=;yGwIPR;y;3NOG z=>C}0VxV?I&+0h?w)kKk=m@*z{_6LSmZfKVism0n!@wUS&%_YCP-O3Q9tIVXXoZiu zKEj>rbmOtyW3-l&?^4Fr2*vWcyKEnQ^FYF@K0?T*Ry*%$VcJGhv1m6`=fOMgp{^7X z0mlhCR@^3k<}q`>0uSUkNvngEB}ts0yq5QwDCx0bXxYUk{CapPXMWK^M0KuZ^#g-(C;EPw403PY)hI0inj* zK-N;D%D{hpu<&j(1%=WL&Dgg?$|0dt%PB)7&I&~UTV z4>StY$J=$_0W*&50sD2KdV5V+<|=e+Z>WyjsjuzWEl32By&KZHR>Mj@f~x*jZ;@<6 zGcabW?KhMYi&EMhnQbd{zi@F_>961X{|=QH%>NhS+ycRE>c5C{ID|#p!-igIE1r@} zvE`0&b1Vldm%PuJ9my!49)8^8R`9zRlCbp_n_}pNS;S}3)>4H=Q$aq2Vy48dZf{zl z3B>U?KRtb-=C!w`6^sf2gndI`nU<&<<%Ts4?7)T=#d(k6R-<@+;W<>o&=|c!aN2<0 z1n+#z*cUaq^G(!}idHm;sH7LV_@2DHa@-%7RSp+>pSagnr~NJsU9N@@W%53{X2ssK zpM6$KEnGuBkpA8=-uft^1Wk(*&Jt2Kz0oYV>Y|ZkONgke6rUS4G<^f0gMTSo^P&PID+t!j(QrP{Pf2J@ld{{$iK>ps`;nZxJke z!e|-OL;GpUng)POM}A1u$Y->}F-*Yv+$c=2DRPl4i%F%G^>c@IFPbP?M8NMs$Nb{X zfFS26axNU&hEg|mAQ~>^N<~d%_;7qxFU-Gb(wdY|5?#S);9q~{(aoOlBwv1h4-TgOn)vQL zZGPM;|3HIm_p3vTKw%^(%2v{4$*HQZfv9?ml=5gN(SQ7&7W@IDxq{K00 z>bU4L`*3PT1);~FebK@)`Q|u~MT`(*%i0%Lw44 z>Ckic@b)E*3H^pt0RwCILtn#vn?@X*@lDI4Xy`6B^Ae>b+a#UsAy_0UT)4VQ! z(V&C$q8FS4i(plK4z$Y2c}Ctm9c!)s-aWXh!0HzztAe`JyYB-hxwS(FagwZY)ud87 zD4LWO6JE#zU3F@CrYwI8B|`D-n0;cLmSS&U!@-Z~9Vn0F`+m7+LI3Ar9q~#6!08+8 zDZiHW-&cFzQJNTM5=z40f17RKasmb_*Gci4NWY1YI0k+8sPD97bQRTdd5`$qpoV`B z|B;M^L??^8lzQL1sv!}*OfQsx>KrMG;pV3{ABWQ7SBre+!|pI~ocrKxBFWD(P~m~< z!^at2g}6D>Lp)s^%PNydclPRYAbxHiZNtDb0c`YT!<+5J?N5|`F`Dqicb8k zHwIIhA`C(jhz**m^a)B84>CD*6S3vQM*}*mww^Y#J3eiK^QAM-xLWOR$EmLr^I9C$ zPYOZFhiY4ksH~I;2T>W)KL>&yR|YUQ{v88~MvSmzi(<_BMIi4R#1kNfiAXZid2FV0 z=ef0=1vgRAO5@>@aE2gEJd0^kz0Y|+YS*Ik+Fr~1p-cpZOwgBaBiN^VqN~z8kC^|>8Ara_(;(=G+Zyxt4!fzzh3WtQ|Ok+x1!3*7XAS95zlV*uU>u) zJlm1wTLlwszQDSTUq1Y~D-`EWf3IAfU;|F)tKYLF_c*yK6AhExosH={TjUJeoc~HQ58zr)PWvx)g6s5lt%QyckQI zd*7s7w!v^r+B&+7dG*vGJ%b=8q#6nA`P-F0ND%l)#yQc16aJ?w>ZBz`-V+b{MdF{j z?7qb@m|AGT$kLovX0FojfJ83wl^f1%`08HwYf9^8?X<1t3zqQ)W@&aaPO@49w^POv zkzFrGu&8W!=%F)vAd@RAWlvud>ds4&8$7NmG*Hw){@+ehq5he3g$D zhhiL2SbplPSzOSRhLxwjIuI8AvOq^w7@7{oj1FzP)At1RUV>~mbxu2YoS=Jw=)R2v zpG$t)g{$X_vu!u*V=tKnq6*zyh>kzUy_EUD!CfvsOMs?Kv*t`bkcD$Q<^e78*wqw8Y15zlC-U)8N7?p{4V2@v44lX!r_#k)GJFUX`_U99NKoX z|6Hi3hx3#Fo?2roLBZfn)rDC2%Yi~Fc<~-<6~9O?&a+(+dzCtMq93z;{Z@(bqD>Xa zTGG5fcCk)6tInOquI<@pr#S&wW!&=o^EGO|oq8KVPpzly-+UHaVn34>i84QYpD#=C zJ08UgI9YOZ66DIL&VnxqECF>$hk_MomrBLL9|rN8dwwm@PNtMeNvF*u6BhA=)nw8v zFs@9ftd6F%wDG&G*bJ5~VmI>zG+Wki0G+>~Hb0-i_M+);^+;q)_pKxX*qRS|iMN9G*$I=Q4XIa??6O z(XRz+v@tZ$N#9ZA7!G+UOm>u83u{lWs{6&8{6fpbX7x5);U#uZv_^O*#jFyOmGNff zZgeObKe?YMH_*9~d7C9~MV8Fg(~qHuyBDjNNxt~2vif*!)b?LhmuLMiQmnyf9it4vSpHui^Aa!qvY7jDGj;;-a_nD>5comaH;)P>1N>YTvd; ztLdZ<2kWanw%hj`wsX7fhj&5cKBsEezN(@?z2{}ef!9!M zy{G?fZ;^8hZP>1giDXGibVT<*eplt@2xUs*-v1rOtE*!>VrM`?0L+*D;Y)Y(voGaX z$Nr;3SNyDwB3$%1<2Wi(I2vdvf!DF@Nnmi+=`Ifgj>*IigpLHiZcFp~i_ut1NpT5A zTVfFOna=8>l|q*z2$?5)isvs?U9c}Drq1SXQG6wZA-wS~bxUk+gx-=uJ;a@15(4s^ z&t#Z&fmg8ot~HC_{(e9?&l1$;cuudeF`j!xJ>hNnDf0I;Baf z1v5gF3XG;pBNU|_Lcn(fVR7SBa0%B{>|@v^{eAtwTl1)?@4eEtITXEPLK59d(VrDe zxs1xoSlAbwwCzvm`Q1W4{qARZNpyBtpAjs=H9pto+)e!k%JkL-pU^WTDj=mYmpsrd zo!Gq}#CxWys#B@3KgM&s07mH>^G08k!oIs8<-z5NA4LX6qJ_`CI#C^NHp53RO`iWl z0VSb?@*4%S&To+g_(s4t%pkBGgVZh|LWD$H7UyBje@ER`;x}G>B~Uf{$<5TR(ZbQZ zbD4ulh%$K!bNM+Ew6`$Zr}y_~fxj|x%Nl1daB#P#NaCOcgT4B=B@c*~_iJSFPZQIc z*QdF=G3{~k?G^(fJqBtGJWnVs67!@46)_*Xr6ldKYjUKFI`<>W1g3Q9A1ClG#cPE^ z$$ykO4l6h z885{AUrFx5yZqO_V5TdX92m}e9IHMQAC9$Y@S=04CkG~iUk)Ds0Iu$B2;)wMuYURD z7XY$3j!)*nR*h$P$IpJ`Hop!*G zZ=*0z;sLkyRD~2fV2+O{-mkE%!v>yTP$?Fn;?QnC>gIL0j=Kf}?qIopYZevuaioJ`FDG+k?&lZYPoXD5S6Y#i+cLP5CWyefwN|E;*L3dy+16R| zfR$QP_sVZ=huD=$rOrm+j+@_plclizL612IyW-x@U{U4s z>#Ots&IH3^CMYnPp|1^0zG82Nv;l1}D$hMz<9SwlgCZ^>a z6`yH={AKiRMm!IKUiBt;j4MMKIC9dAu2~2Q!aJ&%l?{cN4By&HUwFo{GtMUBYLACX zCm&n+WvY=GhKVu-QA7imWJQmwWSUq&7u|KYORD+qMt*uNw?HR_Oo|n~t)JZ!!(?4k zp=(1=xweW)ORKo|{^>rq1GW>%?o;Iy9Rg&_Kt`?z(19wL%z@|>^eFMTYrGN`doOvA zHJ6=Yl_f-XvRWfh4x4?fds7Z#|5P#*7Gm=`@E40M+G19X3wF4@p05SHv1~4Fw1RI~$K9u(3G*8RaOHpMs<%9^f;_?! zCA|8Jj)bvQC?2e(Kh}Y*Zl3xxtE^fWt;+>Hff~VP7*q-u->z=_W2BI5wyut&8@wqx zd40BlkcqruHRYySBy(N0_a*om!>zM{`QK+TS6b%Y`%P6=cy*DJzY8SjN-;FbDhp1! z##(YA(@v=jWm91xc5#X#lScQMXv42D#ZSdlG0dM)@PB&6uMRs%215V5so*p>wC|N$ z)jrXd96G93Ub?4D?PDJa{LLdn9(K}|Sh6AfXoATBfs>n;ug^HwPdU`N1NJ%J__&JP zf7zSvZ;YO@vvxwEp-f3wY1g7>u33v>ee?;+N#5M0wMkbzk4Qsi^s0fzz(_{cI-=XQIu?M0mk=HW@~e6CHt_hak|A^6W_qjZNo zFMi#7RWr`%WwsX73@eguQi%nG&ze7fY!$?JYn6Cy-s=qJZ=(k_*UVvHR(y@9e!>(y z;3aMlOW1e$DMC&1P8O5n0;xXor*L>60nvw}aje7oU2hpD!oTQUZ>RLOoaC$SfCWAe zqn@r4A+zuzS>7tzKyo%I`1hB3s?4|_=kA?SgkBVL#PYwg(ki_Ok^6HisNcos4Fe}G z=nL!U5Jj2^^6)tSGR~eP1y@a(%HZorzG=-bEpThQa=28sUbJt_EdgsxgX@1+*}k~Z zBFnJ}eMZRZ|4^uB_MJtX@$h6|nvCb#FbAAfa;h4WyXi+5&$WgaK$Y!vI|-g1h%2Zc ze%Xj-y-_3^w=~?j`-Z@)%6DA;tU<)ARZ;md1`T%iR2%^6rhqkP??LaMgS;0GM&3Kl zJe!IQ<=Xt6&Hn(yqbL*2>QrV^mA{%`_3n(yy+(ug~D$3`B9 z9FDnk%R%LtID7P#%0ENZ6Xi7_&FDn0;O70&ce}HEBAxe*7eqqAqGhos={Mk~HKEyvGNae5kjbYePf%CE3yk%73*F-}cS(H9%4mM`aM z8YmpBL#CABH-&bt}cSUc6t<)-sw{?k&9;P2Vtgwo^3vQ^AVX zbjY=k`_eU9dp6Ihxnj(uS5V*`lDCPq&7#Yi)qt`0!|`=-qdy&rsPq8Q8t{woooD3K zEYxeiw42UZh#{NTf-^WZw#cbz9a|otB=OFJL|l=&4>(59zDGUJ%_?@Wu5sWX%InVE ze=rBTZ>R_)6gHFhrMf3nvfPxWu=M5MyZAcx|2+}&E+kl!c900zGh8mZ2%6r%D7fn< z{l(}0tj(ST?f!X$Eo6XMT|k&fvU_07@2Xeb8H|*RWc3$8A@Xdyo|_=`pLwJbtwcLB zMv6x-GUSrAh4es17oE0wr6E*GZ|MMD)TJ=CJ{s|JLu=4S+{1bN!iQ}KVHdtZM zsd^HJ*z)``ela7 zr2IX>ecrs?6;}v`Z%G{y-i%}7%i+r=<*qpi8`o-X4~rYmO2;I+5gpFfw=IMkpUlN| zrefh~KKVt06%#N)cPs|?-^^`6ms#Sk>7C-HHRo6+gv;xZWVZOI+^%L(ky9ulP`axJ z)5K*P8jK76(fKv2pp{B)dD(uI3160rV!U6Ziz>WT^Y11DvZ0wi@KVJyz9n5oBTLOR zJt~cO7@MO}r)rRk9SXI1ZvnPser-%jD#QhqR7C@}^yipMyE;7m#>PXr#1*hq-~K6tnK1K;H)g_%sr!%84{ZD?33sET~?kdiL2?KUu&pC6(WTEuK&%^tbcP|Sn!ugZh2c8f`W+nsnT6wN9 zrUHm`q61`L#{a z$_6+b0B*>RSax(LRF!3g+;S~=al@15FSMG2e4kS==Vux?vhR8#!!m=zu4S}GRal%2 zQh&q{XW)03SY_~nq!!!EM7)T_*dfEsgpp&l{Xap^!xIOiv0~E=BPdl>5G2VYnsMa- zb61N)FH(DvqpV`vsUH2wtwyKNSk*6MPZv}5f~r)@$qmmfP@H?x6nq15^cHA=W9*qy ztih9D2d<7tQ5H8G)Rla&Rl@ILU|VKusx8ur{?ujb^tUc5w^9hj>EkwceX%y!1!n}} zVU^+ahzwWW{ z#+C=3Pf7X*wn9<6g%PoUxsvbY@1jY)y1j9jG%>@}2Y=S7!cR++!pE}5yiSUGy9{iE z;TK{IWa2J@Y!Fk&#hYHIc#lbi?X=>&N&l#!4bx*e*fcLg9vd8J=ZP0E*1Wq7_wOu&IYIwdAgVLN@j}jtG9mkGyBLo zb}VZQ@6GkVq1>op>3R=p0#{WXO3ja0vW)&Q;>Z4Jk z#9Lh$Npg0HBuIT!OKI+HH@EJw1Q5%8x83I~B$Dm6Lwdr> z1{CJ$Tu?SH;#cZX+1d~sd_Gkb%OZb#9_)R0NarjrGScB3iZAQ+nwb2X%6X z{B>B$LND~X?lB$a3qi*I0`z|lx^ANotHmq!7m979RQ>)d(Bv9%KeXe00QR-m_?^TW z!Wb9!y_9^pR)=6;9|F6eOZBgzqK4;A8MZR+$T2Xl+yXv*)1y+&H=F0rQZZNd@wD57 z{o84r8gj1cRyiIxzlAeGEhI6gm_+eI>fZIr{C1;?h!p5>NksTbx0>QDv4t)~#ei?! z^j8|(yBXtAD&u%k=or7aKqlBA!9<)v!ccKs`D_8|S(_1&DW|X#L1WZ`r5qIgf|>Kd z@)u^piHMNJpGKG+ppFEf#tT0Y(B50=Z5rE8vi*yqMO166H>JF0C5Yp=vYrJI@5NFJ z34vwtK6^_`L?*zW*tl+i5}I%=#7lg~gnrF~%4>w6+s&C*c9d2$+TY{_Si66D zSgVVE*&D$B$b`alEfQrmpN)b_W<>uOs_-K7PfAAfh-2JEebFNP4;ClJWPogZE=}>3 zMu=dx*uT7|?+wfsK^dHfN$9Q{(7z7Z3^lL69)61(ZLR#-1u7&M)$x5ZFVLr9Qu?IZ zK(LGS?aI8zsZr%?)P#M_JB~0{+FBk_J%r>z z&K>&K`DW0+{WSP8&F=<*QH9^n{p3J%yC~c{hJM=$Z@Yvag&Xd?X7E`@pbA@L;y2G`Oo&AYu&^dU^` zM}bs^cITyoGluAu-+7&@w=5q|62&2g9pdlVT=>U~)Lg}w+zMIvyhACqpU8h~#PLC%x-f zCc`WKtUt^@q_c3SiC}S$yIA|~$$*DvT2O(DXXM*fh$zr*N7>EiBRkq|YJ-SOc79C^ ze{H5YZA;GX-3tz;&Qng#i^d1}oc$8vE2g2s1%3KpC1ZC7kEkJ3g~~GhqDSd0 zfg8`fPi?k>r!FA9(9U@)#1QXc%XEALei5#JDF}I}%cN50NkQt!ee{rCyxveba{RX4{1h=W3eKV~66@eY$yp{l=Abk?WLPL92K*>xkU< z_{bxrx;zaN4}1?-hJA$BcJ2{~MGF4yKo6OX!FP%DGIxQ-^}_$N%Z zcwDIJo_{|fhch1WS-rmQGk%iQ2e+L%R|ZiG#e=I%Kz=P_v7?rpac6|>*N)cq0y8Ny zBI8Bwlar|&N?_1kw67ck&;Yahq7*|uJ2;OYa?8-jfGkH7QSG>*5*@;WqxOUmqwaLO zl^E-w(TW`m4x0Em|3Mbupx40{ohA6yC*{EUv^(2%ci^BD^^eD4B?_SNPpRP{CFC5{ z{M7fU+s~P~bZM7i>#y?dOMK6oLXi=lKmeThO2@4&Y5FsK+mV~aVn?91o=2YpqAXZ` z?LJcE1E*Z>Qi!;}F0)!w(2 zPlRf=`Jv9=J2!ju1}I)^?x?*cDq1dGB^ay{j!iz{y%BI%a2Lqq*e8utWG-6myzuQY zb&Js@yPt9G>Zxz7Nk}0_^*hGu#F(}zOt9+C#Fr3y{56p0LS7r@iuH%gvV|PF=QeEb z&>5cnmBSqcYF2qgbhe`MvglX_BxY#S7P+B9n6TcgRC?=N6`B}cQO){1rYfWY0Ujr)HXW~xnH?&Q-XEb1iO#Q6vtyzaskRKkyCWWR496S9tL8kT zS!HB(A}+5yo;|Knt{v-p|6)8r0^_M*=tAdQy-Ios)>UHuC1vgs-8*nr@KOX z@M?k=xo!4%_+})`GOxd=RyHOKHzxn-_EbgSWHpEs-;LO|fBg5sqw}X0NBkMY zQgb=1Fv%upE0eWP*Ah<`tP`Cv)$#f zjopd4Xp*m+X|a|ijT1C%4jg~?p;9cwC?} z8%CNjNGp1Yh6RTVdf0|c^kKcrtThkE-E-%1RYx91Q``T7)Zn05k;1uePWZ*s&jRu^ z!P;G~=SH2(hB9d^6WM8v9^QK?$M1+Pf1qSk_;v8TjSNu2^7r^3TFK+*0^p%q@ z7DyS`$~2V!nN#p6U@`k$G!j1Z)EqdUWaUCu&VGmIl<1iY`KbOFv@lTHLN`u+9lkua zJ`0aiJHi&`!XNF1SUGfLh$TZ1sMtnTG@>hl*>Y3Q zw-{Fnn3WpubjouXiv~k$A8Dw`pQW-Xi&6Y*>X8gSOZBZ8*|} zzl+07L_$>dwhgAh^Kfu)Xe82xjvHY2O!`_R|5_9d{x`qhac{(I6!xAu@=QI1 z7ULes#@T-OJlCIfMu)10E=o)57FMW3E@_5?S8J)JY5xS4SHa_&&Z6;vyCzfrPd?7o z34*Kokv*LK6}*?qZ1qWVw2`y07bj_c=vgXZ=dBu6)kk>}9B^n0$yzc$?| zg_~x;R-k`;MQX-|Ig1j0c~n?Ou=mW9%?Lfeu~smflP8Y9TQSt%-yz7Qfzg`TAqN!5 zGU@Po#VvaBU$DYBbq0P6>fIU46M41fEazAuFi$wT??Odk?%cHa-v*X*{+5aA^^(lT~bS_$-sK9WwN#t(z*=3dBQ1jC0D;5c;g0D!b- znWyF%etjT6(`@8+MYH556{>*gu@tR=auoPW1LK(|Es^2JUmX3)FtN>B6hl6cck4#M z*9xG1!(6z2WI#c`HP0BCs=@qDMePy{CXQ5p59`+C+{KH3<$wE<>Vwj@ET`>z^WN<5g}1q zfJ^8+3es@Vm7&dBM7U_K#b zeQu_58Iy>E)&tx_YfewTRnT2Iq*)FIpg5By4%SMr;jKYF6>Jx7PAXq8tF1dW=Q){YCBQru?EB}cCkUFjN zZYBHIIxdh=1sc<31&EQ1Gtb2nX)(G6am=b0KTzp$HXKztc_p`)743J4ajm7!`{tay zPCEN*w5}0WWrcJ;Igx&9y{%xF!bRP$Kxy0Ds~$oRvN^rcL zT5}2vYoLFywlhGg`Q7!R$3*6s z=K9ra>~bc(nKe)BWcw-5Iez(*K!pOv=?K)d<)~U8INkf9$$8GCLo}Gu`_3;?(goQ7 z?7v>@F&Md9Kf^@VxAQ#5q4jRR_N+xxB*^$GRlBHfo+)r>ZH0vyIlSY3HFGNBea|Ed zjsqC;mHY34ZPaC}*R}pNIY68+e~(t^&W8tY`1R~=`7hZ}7?WYGKZsrZ;SoK5(~{;w?yq(_h_a%CvHpD4B<07ry&sLnnS2_bJ~V=19fBc}-AoKbr;`GO>( zwI_YPf+g_OUu?c}Wp$)?Wv2SMgD}WH2i4^cqRW{gl9i5iTKmB0Kq8t2hTcq#vx^cwLk)XmiS=%lYlwmg*Q%LW96o}5%(8V#07^sUvciY zMZ#lg#aZ2mPr)F+BW>Q*eE@Q~@pDcj-R%$T#SnhEt?;VwDxuiaCS1BT?5G?@u?x~_ zb6)s(u#UHiFj>$5cRMx(;hE8u%j4wKj5?x5IEL!0;A@nJUdZi+SnFga1&C72^RKYt zOQUsGMxeznF-PsemvCkKDGIS>VQuB`;aE0NmJt#sRb520xs9F!YjQuzl||`yWdyA$ zSQObXlhaL{wGT#P@1r5a$nw@vgo&$yuCl&;pGDdDyj^@z90IyyEl_8XdSvrsdm0(A z$;h>WNwAMqoppRv^(c6JF!ebw^7rPfa}x0V#_LCQlVSEpi3d41v6`QvmrMJV6)4hx zzIprwB3_QFfFLij^DuQ}pI4h%+XF2d_jE5(;j~q>mexBj?2eot;-|AHL#b=Yr_$?b z2ZmDw`H?t(ol~9G>SRi&n&lMd8@jL)3 zatJQM*TvUfpZ~A`=y5+20dT6-%n4iF)I-rpKG)I@{y;)M?~u?mM&%OMZ=-?$xX#A$ z^VB#3bNQeXUXz~&a3s|z)%w1ga}{k{ik}(0e{J?)gC`2IbI^@}Hw>QQX<}dy((2c! ze3r>%OHQJq@1ab9s8_M`M*Ojtp?}`?Tv$6Dn_N2=0i7oYE#1Pfc9J2AFS-H)%WK5A zGP=4(ZmrRgd7K~Jl|h-W@}@}*xeQX@sQI=(17@DFYJRN#N^E9E66qTx&&}_8S*gkU zwuKq^9w0L8#yqj&K1tvfjCWTU>)bjvj&Tdhg>x6@PSC)yXw9B~Xm6orz5gw9qxUOp zZSbj8@A(=gA>EhpK(qBnTKzpsx!pUVX|Xcrsy#9u8}Cnydt&aqS0S*W%&o#rK8;MTR)KA)N9C>LlR-@M_gGgFw$tGzQmVr{K7|8VS?9SNPl~r9qkV*A z5p>l3lcxqSmJ#PvrA~fYg@_!pK6KYEPVKob7CC?|kM(Q5ui&_`f zLTc)YR1xQCiSPMZ?;lVlAN}(?e@Lx|;%j2@Vs)iKowY~`H3;PuhBP~Wt(@LTgc(Sn zcN0@VvocQnb(8XuYsW6f=WK%NN@E&qg1K@s1PKyHFcbM!|4^#eD=Ij9?S9Bw`Rr%d z=3PJ#RZ@yx6-)48+!LRbd(f>@(0L?5jX-dUupnW&PciT|{HagOaQ>hnQ*fHVjSiIn zD#WNM*^1nRI>FHqyZFo0Wmljd$0~(j)j`@hSe~d+X8tST1efAm_!9y#mt5r9Yv^aY zj^LE<)V$?2@Vue*{c3^yQE7R{s<_A0(k+tArN0Y^`M5ez4pkTQ7WLn2x*xH*+Kqm? zZKfTSGMs^v{j!vFzmobs&NECzEqpZ@@bi81F0kfr*vF&-g&$dmYe!cDg#B@q{HJ9$ z?LX5;AmK+(O&Tqp%md;E%#LNqNFrR1Vw9YZ{^zbA*JAxc6^#sNwXfBsZ8C-yO2Smg z;8uwBH-y5SiiY1;d-mkr>W0^j)`k}eEVnAOq-+ps!^o6<5rdSF9gL#0omBzNB5bdV z#3!^4gK2IhVyvJE)tuye=y%i*DH>Nz)BK$|G5m!bN?@7eMIQMbm0Hi1{1V;iuBl!K znY+@m5Y$i-gZqL|W3d`uu8TWBJD_I%I3xBQ7lJg}$B7^Ar8Jz1U{20)jHq3cwz5X_ z5JrE<>e1^dF=8|@|EX#o;=C8c>+`$(it`-W3xpdLUFv@a6wAJ`YfC-5QM}~p^-o=^ z&L4qc$#-CT3H$pc^eLWyst5pi)*k6xCt732Xj$UyoCl7oDg>D#Pk2$k^B=V*C@Kxy z@G04|;6jvI!BX!d=_lDy_i@D}@VhZW9jsZQUA4~z0D60|F;9 zN0s)v3!yGar{;n+4TgH4J*$(3Ea1Sf=&1QHmc^Dzn_PmVdZCerTPhhaw4%c@(W~=h z$yLy?=d`(*VDw+m#ESm@TEKZF5`B0`PVpSqm&GH{8U907SM$ZZ>(k&GMCxuU;C0mp zGuaG7~Msx0uFj>OTr>96E&5O_IaJZSE5 zz=B_R1@bPAApul9w>+&N~DDWWW_=e5@x^E$F&u2JPGr*be;V zIuNj8RRTa}bH=^L>X3vO2XhxzT7iXov6)Ff*3lnSmHQsnQ5^yT8nfRjG?}xd7v5l( z@B$D2n(>FELEA8oOlgN7HK*MzLvHBR93YMZ;gFz~vGH}Id@O;2mlh$`eM^o7x9tJn5z;kRY|qnoM9 zg1U8}=+46{iVkIf=F)-b%CMRBQ47CFcq2uH7o%#AlR5qEfY-4J+msu2PKxJ?{~7`m ziqN5kgN|>2Gx-MrxOI;HX_uPykIS5jsO@TRWDFed^4W+4@dMq*k9FcwI>hWZ(?PGF z-z{#+OZKkEsv*l-1qQ9$2xaP92tY%>>HoBS!Q#Ndb3!M}=MlqqFk5I_?pv#N;r|X{ zdHoIA$HmQh5^Ln(J}qMGj}eHL)3kHtjpMCLMSMIL7-{I&1B788i=zY5(%JsdLGPt8 zpg$9O7;9pD>cIhDu^5x0v9xzKtZ(#P=H-hJqd|n$=-Z!GX%meX>57;?8)}hlTE^xf zwQbC7YH}GBj6!Y6D|Z|10$l+7k0Xzzo`L^%TQIdan|KHHsbBsfe0#C-XvizJtGTF0 zYit=ev!B8VRd9Nz#piTr;Q16j8rZJ@s7N@M4lsvd9`#n%_fU@nu^~+1a4w2@5sU?n zLaHaorpJkHLSy)(zQ$j5W4Q8l{((Qz%ja-UJcwcK6-Fyk&8Oiu#Upn63`G=(uXcE{ za+*2^nNO82I-ata5B4C>dMxBpA}{Y<~1+RtY~ z!wgMs?o3r!QKT%rrZYgdDZo| z>Ffi=e7=XR;``OoYoA@&`vVZR)0jhk7i61yc0R`Fsrdr>qj6pctETq6i?wk86p9Aqrj6y;!$Gr5v;G&;abN|jhdZaTO4{W zpY~SgC00kWeT>!Ht@9-8I#v;WXzm0G$MjEhpc`>2wod$@Vf2|YZbbHL>bRIMC-E5O ziY?oX8DV2N;cYZ>%y(?x7u{~0-?6euYNytiYWC$C$~LUw3wkA>`Q}^`(1$h7at+Gu{Lq z$NRKoPg`LSf76Umzikqbr}g47bfY1$3pi$U5py-gPpbTMgS{NDaO+ULWyvtwXB**t zclG(7*`_qwhzIZE7o*cc(7_YC#!RZ~wqKh_bG~VgoFvbMyA5v4`p%P2(Wb?_LMOow z_hDC*Ypug~R<&ZS{Q6J*>-Xy)0u7$p1@+R-=E#1!`mqOBCSKr-PTc>Vz5`Ib6GB;t51Y0u|+L%^z*p8ECRo>&w3?f8nO@!N zD9zF-16?wvTh|m;2E&q@1Z;}ETcb4X__Pw3I_%=|SfzdY4$R&;@959v5Pvi6`hRNC zwYHREOZ%*QMZAyqFIf3|T{%%xJ;Oay_z+E}BGzDhQv~jEp&@Iiv zX7x^4x6olUqL@R9k-dE+#I*Dx9KzHpf6DPIg4RzB$FP_m6?H|m9Oeah>P~w2$^Y#J zEv@ke$hfXUDfX2E<^`EsoT($R{ML6qzNxRS9^HtX_pB_!y!q&j2|>=EkSCN**xo!; zK_UO4Mmr}G{MkS7eZ6BmooRe!^wBv`;tc!caj#wOb?@_1shdb^Z8X{OsP_#)Z(jHI z_*VRL2m50rF!7=#)W*BhMFKTnFrns{=~v0m!|%BtjP3~1=%By1XpsYJo`jDpV&`_B zOtasP=rr6(8PJ3VZNhzIz&Gu|W}@pGv9 zqqfkp%^#p*_x@LbcYnV~)oaxKW?yvF%LkzHSE;aCX@+{+0*e*J zVUmQ*7G7fnPliXzw;|;deZy{i*>bNs)wU!T|BA1edd8ct0L=*#Ykh6PhL0-Q7FXU3fCtcLv|oChO4!2eXH%V z)m4^5)1T^n+-vJC!`_d_y=Yc=V^{rjYTaH|M=9=$nZ5Eulj!RS8pmy9+}}maa{B+q z#&{b_0=Tmt!bCp%_*b0ZAqCK}mN}RgEva6No5Q#Q9!W>HdQQe*w)NqM8EbyEAT+nU z5MeL>o@%z2_a7&<2Uvaumbt8DlZF5=s=;fb83Z2Vt}LGuO9ksc*Q$Bm;jB zszg$r=LIw&Af7^B^$8>f*gHH5du^6=__V|4cMaXWzs&Z>~k=~fMGW&H-%%P13XLgX4t*wC24hrT*4kjjHJ*12gX1- zzo*{v<;Zq~Ct>#^ciu6;d)q(Ui~g8^Ba3Gs=sgt2$3BrJ%sxedkqtOL_w-jT)DFUp z_uFnl?)VysAo-GO7E1n*VRlcI{m5gBc`W6;i>IeuTlS{2zyFkuw)BeVWLNBD?j(oD zSnh!h9Cy7WeU6b3)54?$YbMxaC9!y z${W@yFv@q)@)*xZz1m3nZaPCAcPDbE@ie?A_;ef}Iub|v@;#-Ge|z9STrXpkdbl3K zqnh~+)Z6~?zBK#%=~BNE$80n40SmtGcJ()V>svA5G6!RQ0_GzgiFz!l*L#<{IF;J; zYd5`}#tGZqX?~1jp1aBI6?f*VZMW!ycu#1VE`9qs8o%p!q&V8d@qYw!B;P&D;NbXK zW76H9d|2~&93A^Gj`p5)uI)JAw&rO(SYq?YS$KeB$jBk$<9keZ;QiCwg?zyqm!?Or zdtAHxyXh-IRp161IhRQ*K*8t_POymKC}3c4R??7-LJz&57B+ed2tJN z-g6FNYUc+bdb~`)JCaYmdajQ2?zVfkM{&o6toW97T;%nI{4e&v83;P=MUaYr|5gOg zZp8aNBQdze#G_~=X=Ww4Pl{{u2Hmnl7ZV6S%nu`qh@P}S{4 zaPHk}mg(=qIAS-_vSLTzlp&@+a81~&T}{3y|M=t?+8xWlAcM05rAH6!PQQZy@be!0 zR{HtJ9_mOw0#C?AdJym7jI<2zPmMGzzHLNB0ejU`u;eU*l5aw=n(5P9H{xBb76iQ% z?<2kIjLXuuProe9_xE=)$jiR^I(7gvD9Xe>yA#LgGL1riGV#X~ce*ltZhO_|U&VVt z8GL0r1MePXV*h`8>80tGGcHN5L%=*z@NXL62w?_{c|0-G+Jgf>4HokIu;Wz$ zFcETb0FaAIe9B*t3otA@Jjnt|Chu_4v(5G=d)0Qf-~Yr&T2E#Gc=R81^b=(RFPITl zhf3ki668rp+5$>H`Dh0racmRyvJI8L)JEBDb)~OreiVPjo8`zSY`4oKEdOMh@tH1E z$OeOIqh!-e%Bi5_kW8|zUPV?#TLf8QW6WR&e~Xr5hWz*ww2WNj2+LP~HrmVpE-_Y+ zazzx7wB&bxGp{yRezG>1uWEjF-^rh9cg0hU*`m(r)fls=`&G?wKhq|)nRc*E2BV(> zQXtVbhLll-Sa4C(Rj_>6BH18!lo+Fqm8=(3-;tO7#ya|q1O`QivQQw3rgr$o|CFb1BmG=qJeJogyfm=qrb&mtIB+SU6y^%E)98_>98g!>rn&o-e5A0O z$43$^v_ILx`@`jvxcC#RXv|JZvO4CFL`y(EgO+u9sN_E(nIlrLpv}A#h(r=kW~bA# z4zwmh%3QDMP#Z|1e+Eq!X@ziJiLpHWAbsYaxU)1@2p4`L9cZN1X;BudL#fc`MA%t< zE=dV(mrRGqQ=j#ToMSsRSMG5tf5Pv?YOFr|`A;>0}-B!oXH`3GA z*3VDT64$hoJVl+!WSz-ZrntyY#K{K^&0%tlF2bSGVY44u=Kt?i;(OPIBhYgB+s{*A zj_=`&_qc^kt8ZSXAO(*jWT5T9uIgl|TRw810!dsfzu*l^vHJ_VA8)V)XuboH@6EgT zEoW=j4&OaF0SoNZ#h}aDZ)1I{*>-fxIM6izxUgO(`K_#Imxs@pxMS$vPd}0-V>j%j zZ#^ds!7fj}6O~7SG9BD=DDhawNWJ)EJ9bMkh`@mHl56ItU3k}Lq+Wf(#cu9gWI*zg zH=nH?gFJ?I@Am2;_mJ5uJc8pRzx8+bXjkk@-*>r=Ip*=m++E0bS@IZR9>+>Qj2JsS zZNL$${62KoCzmOB!6RlP=rbO>Cz;BAWPfqj@5qTG^&ZlvH`z`iwEaz5aZ&JTZOwfxOR&{Ods!~9FChYid&ms}027FBa_KQ6{Qg}bG0W3ktJ zw_CqL|c=g-c&HoNJQB3>G8 z7&mLQ-W5&MIS9UnZr-QOG4tSmtyG})f;XIj`|&K@XZK_NW9o+ELn^d$Bz9$UF1_{d z?@hx-4Au_WA;V5exBkO@s^5iUvpH`~#DhOqyz>IRU;Flt-mfE%xkI*-A7nsriTkan z%31E-J?-jwc<-yN(PzZ>FmuwxNz)qLQnzO6jOl5|j&1eIeyQ()qwv5RCd^2`HRn`u zT)ynqv>ERX{ox7I)BER~rn)m8xG5dv4+5k~*nxN}f^AH{y5^DenH^QoOqG6_i(^!; zUviO3-RQWZ@vrACNtcb8lo+f$_ko+c9;N$C>|y}!wxt*ADA@nrwkiG6I{WZ1?e3O8 z6~VQk!w%A3-pwlI{Y&2i@WA--6Vs-R)x#X^ zoWjq|554fd8jAM;KWqte;=+ZVG_Pvg(q#tEKv3Q(+QfO;I2Fiv^U$`Xm^bcg~}s=0SpIb`#aDTEVXRu zZ`a>ePmRe6qAT<_Nz^|0t^U+~!;kcL_)q%+D!lq7^OyC|B)v_4H!Y}_%^vqp*={GR z;p?nbsq}L&RM>%YgI;7MllqH0i7LOA z^FldZxwu-RE9VL4ukfKHl?^qBbqJ`n%qcnmy3~LQk~~LQ&a-C0__^(s=p#E6(ZZb93G= z`onJ_kFZVROv1$(n^9m$X>iyDz5&E!$tAm0mYg9lU|n7QT*}g_OpJ2ETB<3uc2w=D zQPvA?7TLS$ZSNoOkgrYZD6T}v%?-bI5LmLcQP)z{-d-uJ(RMJu-qFrh^%g~x76r!k z%;_#H(KU@$XhD}15?8ONx2G=fAHM(7u!i)V(;waRzVzsp>Qo}$&(%B&zL;MK+oaN( z8ag8^Q@1nxQsYpeHe6fLxh9K`&yA9iE6f_W_1?(LWu_9O+C~v|<((DtsC^E#bVn)I z^DXjP{8^W8^JyDFQRV#lej9aNV136Z?C1BwHmSwVplR6KN>|z+OciXUuW)6KbV`{a zXHZ>@m$KG58hj~T_wGk}`V4rs)*MGpu;V#v_B9GaM-A=x-npT8ALXIFhqVY^va$Z~ z0R*7OCP z2Wb~A%VlczqXNm8uX4YZ#K)NLWLG7sq0_Ow9dRA}&}%y{{K+%ZLY%D7>n`kG<@T~p z+eudU?Q@;D`t^Me^ymQ|ult^v3q{Q%QyCOxVvsCS1VvfiRV=iRAJYB_0tO)e67Qg7 z;_kxhpSG>-s_@G%O`WIrRmRb}42r&Y?W0|TqGWw0cD;7O6a_^o=R;U@J>Kaz)-F#U zS$JleKYUcW$pd}VK_9$#-J{1_o#bK8OFxbGxH8?aV@nQ-viw=>3Oy-3n)YI6Xs4hk z>HhEZh1sv4*=m9ANxsXMc<%Hq(_a4~H`aHTZorPvw;&L_aQMjd@5S!Y>AvH1GR6{* zUS?vD_~LOB(s?*q_*d3HqPp3B6>0#{L?Wev6j=Kk9AqWfp z_AV|iecFOWUg+X+fkhxP@Lc#&;4%Vf0iEC8y~c&0a(SX^fqui{lHZ+}QT9WLrZI7` zjmh4qWl$BYFZgLAS#&%m35vN|#ViM?GQdG3lB*Mro#-QVsX+tEOK z-Jf2sF#x{YUwH6Eaza^)zYS6I5Bwerrm_bl0h|7g`hb*Ex7*~)h0KDmb>W`e?&ksg z9pkn1w~c#~9Rc^6J(M054x~`OK|Ntk#5sbl zCw~o3Lg#nD8dZs|>@O{J5@!-j97-YB6&cn1^TR;nHO~(*p7VHhKYD$ohy7HQ!atlln7WQv5XW$}16@d^cbUU$0)TrbtqewE!XL@{-pALufFx9CZLbTRuy$-g&++pBrq(7L*Q z>WyZyUe#fMljr{Tdik6BIBk}+kq_wnHQ*-J>N*rcjNcB`{U84F4&_|L6i~;ms=_5i z|M@^E-z_S}#-+8rb`Mm6j&w%Y?q=RNkvsH{aLe< zQcM4Up~X(q;@yL0y4(h#eF|gta>D#>(b7My;^jxP)ATeuB>YqMXLk>{oDLxF=YZDl z@I$nFAbUe@(E9+&ey4wctNOF-G2*No^J`6=9AOHw_?Yh@RA|>1emd57#8&+olS&nd#Z^xPY#H9(76Ijb|0iAzdNI_7b%{-2lI2}t{j^lRUD?#e~8gIx~{*>B$DR+#r zvi04fkPmy?y<1d5!cK=a-e{-wK%DEgHAfqYGlEW?LP2&*gvp@(nGh{Wx0mnI?XJXzqrk%yoBe zN{?LkSi0~>@&3soFxrz(8Ue7<#+J4_vR(bYozJ{|rEcciwUzY@PBep}tfzkt@|~k? zQq$&VBf!zgt~TZ*?(*FF*bemDkb{Lud8(>4&(siHpn{3?o>}c25pX1o3KNa z*B^zgJ=)k!XIAc+X1_P_$D`Z~2vg><1V#B!1=Ag$cu??}TspP8S>LT+yXo)9cNBft zP5$x59aVpz4ejR6XJgJN+u6>>b^+1g>4a-%XS3f!24P2B2S1d&-SVSxtogV9xq1Mw z8|`-s*AI+-w>GGlE;`bc=t1FT8}6=7Ps&{r137(UxG}&hqRd%Ab1!- zQQ~$SKA0XT-Yxo13(ibKQWbdJ2N`d9{F&XHZ=W__X56)VM_P`fek1+BxM}INlcz^f z@Y8tD%^p~lg`3~2vb zaSU=hJue$OCH>^dvxWM^Q|r41P}}iG-8Xh`OZ+R-?YnlQ=?HpmJa8cW?dCN&o_S>Y zG3a7&`_*{Y=_j^qNcZe}I_<>RyKnc?Y2k=b=^vh4n^qj$oqiBImicbh;do-}z1R)= zqpR;NMfdZ44;&v4aP^1@iwMf8jBgSuaZwA4A6j6r_sDT^-8gLVM!O5e&w4B{VNUhs z8`MIM^Or#c*GMi?Q$khRMRYW!NVdrCx>3)?AU!FsMIq|u4@z6Gp>6z;X;0F7z2og& zmuy=ZxTReGLCVFFSXp!Zv?n|Rp_T%Akpy)Y8bKz@{`Q(WarPJeW(!wr6O*@DtNtj; z&uo9-MN}CWhSDBK36yGCQ6QBq_zYe7O@Y~_zuoWjTT9@!SlBYZR{qwQ%6Mp!zo9%9 zsp_WD6s2$^V zn=P<%(bZJ=)fxc3R?aXCQNKf078(4ZAo#otz?yJ-bKE$!h((vjtMT_bKh(!7$3k@< z*!%(u*rr&71x2Yl3ejw8*ZfHTxW71fTE{CXkwRD&;1$_llmj}Jl?*Bf%`%XNCJb)@ z=K0+0&^w?_fL&c&saA3*k`#p~R@g%*cCQHEc(*8nqjnGZ{DAK|?cOb#{hRyC`83`u zcThj4Kn-^(q)}Ch>Cud>>T7m;HQA#%KD`zUowM0DAZalH74CV*(qL4EqJSA|)Yh5Q zSx98IMqgGK!h#ycld(BNeiVe(rIL@LMzupA+)l_<14A+I`)UN0=7}X&a6Cbs#MKncoEo`?@?o8D})-f`6KEyBltT-p(jtYVPm#Z`lzk?{t24mb1ftiicXL; zmHmMuWUA<_Y**7=wXDMFUBYl^`QTM~|A-Qd?d2N6zQ35vb~-9;|I2b1Iy|QEyP3G4 zeg}>ezUyC)?j*DCrj31Fbr109(J%eRP4&ut>3_8c_~6Fp-+kl%Y9I6)|J?Dw4IlXS z3D=l@VDyCRlIMm&{1W_;`KPDT@ZM3TUs}7Yw%d^J)M%UdOT^P*{>o{jw@sOyeq@TB z6V5deLkiAdFt*Qhaw18eFP zjnW@IxjOML(pR5O=`QR_{-;IfsOrjblN1+DnY4*4KKm4x6$1eUhMU@p6Br=KF3_8IFn)O)aZyQn3A2iGyMY=3o zPNX!Z7~fWkvXQvIqw$qmNh1d>D^u3uh)|^>FHS(f6#N-))m^V9)5dWpe=AA^DGXAD zT4rLWY3L0_VNaC9ZZU@dK)PLWydmN5XjtwK_q%qYKv*?iac`mA)E9a)EDc0H_``oe z7jlB!AQ?#E)*+wm&i04@j5Lqep63TDaA6suCOb~*TkJLt+EwLff&4^!(NG(`z*AOC zs98r-rCzq*lZg6gY5LT>~3x{~Uid3ELZZfnlTelaVkD6CN z>vqJBC^e2Ee4N>LCEbTRj{sX9Xyb34V3s&IFVYlBW(01KRv!Q~prcB&1fJhwA;?&TZXHD-F3 z6aES4?B>>*Tr-MyrYSdA_YX1k_D4SkEV(IB{wIq&B+CngL(?k`UXV;r*`KV9-yi{5 z#}vbLkoRBsN00>W5BvT?ch}qDXtR@ruorn1V<1sh*JP67el;~N1bi+2am&Nr&ixs3 z>o5ee*-D0v?f@(`(#t--`aq2xm0@o%WB5(;ePC)LDbgPHPo;10OIZ0gWTz^G;Ns;OgJ+X&2`Y$kUAk9kd%Mc8bRL zihz0CiOS7_a98cVh^L ze|&s6LDi3$q@Vd7c%ah`qVItzeU*I=^gYn`K;Hv>51ddQ=m$nmsP1^~81sSIOVXuy zuP4)=Y*?ADMIfq7n+{ZGDvZN>KpCuLnmp8w31I#SKX#XSrq^Sa-@9fkRQ^UB>-zqU zE0u3|;qj~dy8{8p+jehHA3NpTGz353t{FQyeH!mAefOGW{=>$&4<*}mdMtKP!aJwW zlc0?U_NNun&RJqevGc$T)t8`R2FRtw>+?SLo`oRa{Rm(luXHJN^0-;1zi;do9Z1L0;bYQA7MvxY)5bTgdH_GY{D&==pTsWL zNRx)tr=f9!txUe7O}FgXp?Ar$uX(I)xAe80Pbn};TE5$MtB=8MJbhf>14rcnE*klQ zg%%XJh@yoOgSNP05=F`N6Bcy=aA;MHg%)GeHIrdcL{ILcLUmef_btN1?2N86O=TFgif~215y5DJ8V}+mQgm zmv*yllvAegPgMJOjUwlVyz{=@{17`r0p(AJ7SzPOgJNryAq7%UheG)(dy08*M{u!2->q}LYoBsFf03lZSr zghYiqOe5z&tFd4#35`5w>I@KNG^Em90ii8TXI*@-09v57kXtATX{w@q8gJ6x<^lq@ zmo1?!43^jfLm)5FncFPROw+-MItqd!@NSEMO8O6ck`W$rWCf@;80uLxNe{g?%q-?N|wlZuVH zOxuzPLX9NoHz@ybo`LU!!dR`1~XWn`#J33Dv-H9H-pKF$sG`8K|97_&Fq zTTNf(@0XB^a@=*5NNPfoUoFd50r9sm-!YoM9~!DQSIACxLp6c2?a<==?`#5je3bry zp5SfYZjLFYCbXu?Pni*!?4R(U!)&KX)0K6ylchXT3W9#N?|~D`1N<&FcIGG@&)e=8 z-cHu?U7$mT**$a!@0vXt-cB-4l=^YA#-KfibTst1nPYH#=zhGHc6i#k;rOjJo(OGy zF8ASy;NRNV*|;mszjS7L>{}a71k?I9^gU4XKs_Q+L-h-N4;-5Z-ivoizH*|y^YQPV zd^~*wJ2#u@@q<;M@T?JI^AW2{hFKtv^y?60jC9So$?13JoTlPEyLYCyV+X4LBnQh&DugOGghD}$1Y**5QwQP>3=?|I#DunL|xbyLTO7M!8#KUlviebbNG zosPgdWioMREz_@0TaYdvKPj!*w>Q15IL>(yj$4kjvlj4)+CJ=i;2C;=3kh0u;O-v& zaKeHM7gM-k5wOH#9O?;iUobBNk-;yKT7rQHNV3t63==A1CmRu@jU8OJm`)llmXOq7 zZKs?B(l-3(q7$L0xyWJkUIegm##5#iCdEt!3d#J$G1cr^g(|nH?r*b0_BdVu72&76gTZBnf3m;bf3^OQzz+Tn zIix{ykfGY&#oxhG1=l1dUnCEg+Ja&zp*|-tf4iTpM~KCr`#A*oIF9iieh)vg-U6Fp z0`3F)#5%HS7eq_C^pmze&dPCXOoq>NE#oxwwL-f;8uhN%+HW}7(~b;?Fq6$tiVBBO z^k&-l=e?l&gB-?bbnvt86O;nFgQ6cuubraV@1@K-d~t@C0v zzt*gF7eNuHMOSFBLD#7$PsnaixQ(4?YxQ7M`y>FEY^YIH)eJ-s?KN@3I%Keq{VWKY zdLRglA*jJleH6h!hi*~p*vt@YyiXMhiE$%+G6 zLxVz{K)q&nDYs^E*cpOBPElfO)1cTX8nFIuQ82ih1Pgm}b=Vm&+i3=5x|$&1G+t!T z0GDWpji9){%}*Sgc|<`DXf$xD{;c%7nh@tO`q6^^Y_9u1{O)$!_cq{soagTo4FwJH zt^Mi#33o()hUOePYY58jg+A-gxTESY=;B@G3>4C5_Pq{1puI>9@nMeaH3Cw=8~Xn1 zP&M)W#r+=s7o=^j0PSR6L3q}&-9^d|2;*~-E#SLFmB=pDCO?0^=zEo!FLdY$D8E+r z>U$_e)jmfgx-K{S$K}h0;u2PeU-b|G@l}P}Yn0`E$&pn59s^Q$Pobip?R((6;(?(f zhNM^i;T7qjFRe_EeC6?uI!8_%mR|LTSEh%)url5K$=;6~KJ!(J(&R-G(%@l()B1Zi zrTahgXh$8#g3#*`z*wCIax7AgjfehNee?SCtqAW9WlK$mQUpzK} z$AL@!z_b(JnU90W)Ts|gP8g9ET{R~i!h3cf{_^UqYUmO6%`_GOhU9 znzU!@zI4`WPfODl+oqsQjLfnxud2PH0Oe8=`h}jyZmd9r`6wVT`wo? z_}OFS_t8^ErEP0>q`N-;Q0n;3UdWw!_Q`3{m2=arAHEmw)U}OA(pg-d|Bmz1h;hSm z4c~G6&A+-!KyE;IByUm^N2!WBcCfz9#O$y~ppJ6%0^6Z_EU#{OIQO_yO1I z?_u~s^;45(r8i8N5x+Up6ujT_Q`nt52Je++|NhX*yR~ySY%k%RPa{H&^n;V8r|Cn6 zEC0@^a|L<1erFZDeg)oj`m>YgrtyR9y|C*J9!T?tjna`*Nv|0{O?Zx}tFU{RY2AT+ZO4<4{Y&2i$LIlkH^u6NEs${6h89}62%&|N z3X+bC6oQzq0!l5)TA5}&_(VrplH-th@f)qtcwLRZstX#~7`&3{wg6!?fYMDP$zMU? zSrk7JW+QWKBkKj3k3YwW#xG_A<={_x4YGn|Au^6_&Q-{>#ELc$I!z!KZp;?*6ZzfG ztY^MxoATaf(J?Q6hrv_S8NbV+Eb&t|O)x%#tg(8t$!&r8rpxKoNT;Jbq~H(R&2F}h zNc!8M`?{r5Y5#ZMXjN8p4-RhC!v z4e84J1Jv?c1&u$(Jxk;P9r(YvtD&*GxPMZL;61dkyJSE z3$c=xM!A0qsA}~WttMS{FKR3lZKn{@Tl=ZUlF6{L^oWLZ_GdCtq%2lBB$*Le&WbLT zf{fTn*nXo|HIM;S8;Sv?&tc6^qN#i(oZ8NITfHFj_$%uwpHf>TgIVDUnDD@vi6b3F zRnTk(UKr|7Ms1`VB3P$-+8_Z&BbmqsUeH8~PgGi=sz6w@2B2+Ru=eM?3@+$!*Ok$;1?9NM^vVbl(AmzFq+3tFb zfz~q_xf*3yaQNP z9po6U5=&MWZ)QnMX(wq(ggX8U=v=I`ACR&u zm8MCwP9q%Ukj(x?PK>N`A)>)0Am!?>)S)VyPr?Q=L?0?oCf4zvplLKQGKASsPg3Ox zm8TzA*SNdv^P0H$BR&1(kSxp-A`$8j!c<4d&m{`i=Mf=sl;==7$w#^Vy{3Tr%n>hf zWUoHw%wCQy-Ti{fqa4sDbTC4vBp9^5r(O!JzT;JPvW|9Ekosh&Y^F~8ql`+P*ExdtE-=>iN}S1|E>e+(0*IHJuU)|LC70E`to%7&)O@qj)A=+ zOWB7EreE{E%i#AHrE?JMoHW0B9onRY6Vgk5?Ggk)C#QYe5DZ0d^Rl;{E4hc^n}d4} z<-gIBN2F;8qS_={SAFL9og&$Th73+)r;bXOz4hEQ^PH3GL?{g!Iyk-Tw=P$(bN|!( z(}GK9rK^AS;ubnqEPCOb^!mShX?o!=UzFxvI6a*-#OC^T#PcqmktQJE%78xq1`Rga zQIkfbtAFL9H09KZj6SD}e*BEI_$BisV)CMi>56w=kS5Hs%|K`1p@8$>xFq9V{ei2} zD}MiqblTPP(rCyjQ;Z!3bZ`Cpd(#y7kwMgc>U-cs@<4yb=!w)D|HEyWh2T~kb<3vw zFoI_ME7LaY;=6d+t=QFhBz*+$$^5w)^V7L_$K&0nUnVl{GGyY>tr3j-Al_3My8!vl z()-SQzG%wJD<1iFx)Xu0`*uIA!00s-PEP!bU5TV!gWZ*1dAfegZ7GBKqp^$h^M;O0 zSKwWiO#ixNgZNcA&X-&9?#zex>{8%2+E%J~2m#}}kMbR$C4V!|;42gT_fK2fA9wJY z2~*R1W-Suk+Jk$rYja2XP3#t)jNmBKNCZc}y!ayJ%ggVtUm=+v-1J!b`Co6Hfmv_1xx`WqqQzWRd^7E{bps@UGe9qT3w+0$b8U99{#bNRMbz|96zPyde)zaAdoJWMMwXJp96x@ zQt5AUSh;48vr~NdE$|^F`ave3GVp@DxVcnsNcT?&2*2fWhYVz?IhnrUXZL%kiGIs= z2g1d(SSHG5a$sho5O&+0+v@N7PKySf{RKs4cf$ZvS1Am|ZbY`*>4H1#Hgf1w1{a6I zGw%$sb1Br(FNUuBPvezJTjmF&X3PFRdv6|f-BsR)9SNZc5QsrSAcQO?8ygve4W4jJ zVmpJ=#7$$bX8I$Elh`X$lXkhOSEiM?X_wQaO;?;O$8Fk`p&9Iz*-mOa1ltK77z}0> zh)Ez22qXbQ148``&+pm$oO|E*du3zkLp@u1&$IV4@890%-23f)_q}w=D+_DlDA5kz z^Ca<;udN0?c|H(7bD=4qHZ+blH1^3y9bVy`h%<$@3@|Q5Ep4{kP=|R_CHYI8ysG$l zPPuWEA5`a)qU0Um&9Bo%8K((o1igB`NJx44P2BcVjGFGwhrCKy^EFP6(Kw_``L1a@ zI%a!}2Y(4KapjSdZJs6Y3gDy{0LxdW(>ID8==A~?dHso5sG|}$_=OW~Hh^~Aavc5k zHv|EB1sZ~fb%H~61-5O9ZQ7+ZBODFr(=3$QE_@a%TxPncU`gH!p9GYIX<)T_|Y|Saxu*b z(ah3|PBXV>ra&=H@uo`(f&cMkogHNaf+8=XTzwQOqLH&M*dHQ*m zM5|jvu>{=oV@}#6#l!9Jw?*eX$1Dv#x}TVA&7bNnnuYg@CL87l53uv|m-fy#zbzV? zcT1!y2YAjJ{&xd2?&p_B7guo|r{Wf;9GkeetlMwnqxOKW^#W|s@k(utjH9SRxC8zA z8-yYIk;o=?v4nQ;BrUbFDj-)HW;cX{WJzeiu6n|!!Q z^u@pBiFX0Lw-#yzh;!aUhpl~FOPrnWy@nOdCYR- zx4&R{+mGG6ya*d=x8S7SZEw4M;aA;0_ZyzIyz{5uzg!I8*JHEmgTL}oZHj#wo5An; z=@00ueYu%;l8pI3{_4jqU-b3QTrR)z66JgAo5sNe%2Vz^KhO15!7u#n&r$Ao<7;=X z|LSKfAAZZnmXE&U<2oToKR4lY<)=R4gB*GuV0{4xT*G>-*D~n?pNP}XWyc)X1)4_Pt~c-w*w=dl6=<7 zpNz=r(2I4yd3Xoe>;c}nv~|A$vE+=%^-ANtM9b6Zd>I14K#i30FOCdEYPIicQJO zmTO-0)UZW4zb=2uWy_oY)lJK#PxLzNTaYj3)LfqPjn}F9>i_L^%g1lNWBK;K`z@HW z&ywDAvGL0CqkrLNmb*W8&+=`5^OegDUw7^DI-Fp={3(|#cYf&8I5quIY@}W5dX{Ex2Sm*~e@PVdvD_^Q;Dzx}cI>)&eM z_4MoXwX5Xw)a4I+mRW8au#KJg2@mUwWmj0pl{>t*_pK-(TZC5^BTG>$j z_jla3{MB1;KGVkKS@U9mKZX;sDL*vj9Nw>%;#(C~g`32G@I7x{{^B!lSbi%uB7)bJn+UXz-K^FV?0q>;DEPU_W)jZ*RWqzPq&<`2ham^q%_KjKN`8HAHbsB@j*sH2kl)efw?A;dzLJ-}vHG^pj2F_2h!_1c8ZSpU?WA6vfnmhn|4{Mz2{z{%CWcI^w6mp<-^y(!AC`Ta|`+_e1U zC;e3q9dNJg2z(w8&;kjbU0P7!B8pb3vJ5gPmlOIC$Ia2H$(wpCw4kwBU65@VuugIF z=!dJ_*>DBrW`#c`&Ib&0I;1Q4WZ!-mhrZihfp7t@7LioP#S280H)L7Vp3;TVS+4$^FzU#`Es$>dCSYx@j?XyAuKW=q$))N)RszzF#w&p;;m$k!DQ;!5CW;4E;+K?**Y0@6iwo7jvCBE;YvH|m z&cI#^(ml`NO!F0Up%#hho~fMne2{#b_a^_$vn_r>&4bBcWf&SDG66YXnGde35C?cW zZ>YK)j9Uby7iS5d`#-SrIU&4_b3Z?7K66u4a(<2Otf$`=#W;aFZ^q-eQ4eJkcVoudv1NxwD4n>M`Z6AAbHZN-i4X1wRo zUG-P?${aOl2^hpHO0is}-_n<6*HnQLN8xCptO52+fFIrOTW$UeZw zyo?iFZHwKnqC30<(3bOUo|!oIHv4Tei6Svgi=v@dHFr?)=J6bQWwiAx1zbo`R{F)s zIP%Vp22X`@Q}p<6i^iF8>DS9m0Bg+=YaGj5;VWX-A@lPtoYQGWre_rAVm)uxVca+I zdO|pOPhY6;S7+l?#gE30`P zM}Jnuo#oe$ic@zQ!%bn(7-eAR_aP%2=p5i_zqGL)94kFt{A>8y|3nztWq= zF_8u>{eyhXaGvzzS$r73V(er`;9)@Eg2!C2e93RSVY%q?i}ZD-U-R8xj#GEfzy{y_ z%f*j>?DECf#QEXxdR1q_O&xAZa-;Kk*d+SlKlbC=XyXRag^yj|wLkU!zqq`qzB=~; zyr}RuM!)8}Zq%v1+uw8N@*Hd^eaUaVVflC8^J<;^`^|spYnCTI;|bal3%!1)FP^FU98KkNojhEmuDKiP&(%DkyyOD_U><*;~|?`*AXn8>3g^ z#2x!~+f5%|F1zwF)!9zpuKBX3EkB73Jbu+L`Nw4Z{m0+-eXqbqAo7Rb=i)0^f9kLQ z;6E0pJxF63u_~Zs4>-T;7K4D6H|LGt8_xc*z z-|_vwK{Db^th4#|j2B_|CWG z<$u^4h1`I=6#Qp><4S-6qQF?&2V zHn%0bkG<2nAAc`46@lp$Pk-FO#^^6I7RpQizzYvHMxXl|uhU6SZj3$-IkP+qr%JDV z$<@nyv6=cFd|mEm?|Wdm+N{-b7Px-JoUQW7~RgD2XLRTT#UKBmmPug34wh^o=>Ck1)j^z_)5~}|H{uiq+a~M z?eAascP)N1lI52AUC`OdSH0~ubMw^tUDyQs9&C#Jxm(_~TzBy$cve5SyzjpIuv}CN zMyFa2;aT#c_V?a;lm3R~ufE0;lBeOv-|76*A9(xnXFl-G<=GcsjGkS9Q?U0wxF>c2V)^LC+G4>^Lc@r3)wB_f(#eRF^_#p&%7>!j9c+qHFD6um=_+r^S;~<~ zEkIgQ1~DwE{g-u5u* z!M=Z^RC7TvSS{mLVUyQD7=j~slSkbx-mw?OrPsa6e~9NB?Fsz`Z+l+j9-KqDIm*5i zt^$UxRngSC5yUZO%fVX}sK5`mJ!NoHAdbG~+#tl04J5PvS?PBLahfqYjrq zouo!IM*xq*!2Kk=SI=wAIj>XG-@1r{3&YRhdcKy)zJgFJKGj7o3$hf$s zO*Ar+cunk_^jmKPmFYxc7V3=-j@O7|b+gd49nlqgby#dMe6K0u}B1 zzv5}Wp^xpiCVu93%zIo;z$ZBAF@wO}ox;Zxa?A(j#van0N_W5d9o*rFIShQ}1Q^wA zQ~O)_-MTNo97pM;gGDorqVmrBJNUqv(`#CADeyYawc_VGj_)Y`wy2z(fKRLs@q~oJ z5_l7R?)L^C#eqp279J-NqrwoLQ4WLt=XLdjd^>956wPnVX5Jmn_qG%sneUl@t9dsi zJ;Ef<4rr~$2maCdU)6ZRlisJwW9~51;7MGFl{fn!%ll|jl#!>Jbsuy1*l(12eHyrRPTGGUka_~O0qw=FKZk%-=Vs`-+f=|{_H%$>H(jSwk$1j z;4Ru{<;LrKUW4C3{X4&`jo+92o-e_P$niCv8 zd3yB?|J)mrv&oIW>jik+Ke)W}RqvbGT_@)J?b4L{KXX4$8~b%7JW`$(X1V`dy=73F zZPzv2Qd)`^cWG&h6{olqC{Umf+#wVx?!_ULV#SMlg1fr}clY8N2(AGV{NsA>`+C0T zeP_>{$4t)2zwGRNtiASHv}4ubT_)_O)~NyJ4bZ*vTai}X*?W$r^UA@)ECsr38SM=_WY`HWZw+4bT zq6rk+{aps760{UaHrzXnC$xdt-fO_`&V^hS5dWxc{?S}`*%q8`q6Nry`?w4H!U!v zEm2?hngCZF?MlYs*Po1g9$7Xp$zDLlmD@snyf`-88Z| zn%QABMBaw0%exn>l0Q&-xh5~?d6m&*{x)RIlhw<2C#i~B(8&M3kJzjB8( z2L?mKTL!xodU1jE=F07?ITo-#y}biAQr@>zO9sOOFg}q%hC5`BWDm-UB{-SvTni9Z z6gAi8YLy3|MLjG2@Kq;-(7t>2Ji0qEK*P?c7P4d7n&-pLLY*YVR?rKs>FZLe$smUU z+5uOpd>37RU8FJ0iXwmIzlblk=`AIu|MAH-RQ2PhfCT7MKf3&Hx+}1}ku5AT_9M?F z>ii3}r+P8qa?&*G);j1R*7h*WQ_Q{f4cbCD5o)sZ85WJLzmmJ-8TIh&TK1;pG>O}% zJEaE-cQ(kWreTGa{=05>Z!vnk>r6qZx$1Juue=(}-9OdejRl{@#%4)Vjir>|s^8p- zi2W&t7jHLqGO;gsj{eM89#?<=rr4CRLtSV%P{mYm(OWvQqcakz9{!TcP zY5Id7(wX*Dm~)+^qDS~aemVRfH!Z`rOEj~EZ*>!GTwDf&Jw_QnM<|!6DaA(Ivd}uO zHzX;s6iB{$f<<>B>q;sMDIpG7{{bJV!g{&t`gvqV=55@&&}fU>9_xa5_}>Lc9~Y5* zY2s<}GhL|(?Fc~)>cQGhv+!)$6qcpmd$Cz&Jcfru3hO*22Y-orMZbK`^$Ytj1YN?T zg)rt;sD$i=#7+FLF9v8<C$;F+^k)*Kl16diPKQ>_*?w2_$>mdQOtTtnJ%i7;R?jg%kXD zVQzarGC|b@p&U2<9FueawaHa8)U_^X>nIGg?VL#F+ec;{8*|SNS~4-OH!t%&&jWl( zY%9M0fr7La*bBb*8pOT78(uwagQ~hNRe1GA1h=_(z|k}M%N4gU2b;wX%02HkZSVJ% z0HEnO;MU+&;O+IwDr(gyUMXtQxAh1`GuQ9^o-f9C8~D(>6^mLh0Nt@a(kh13fG2M& z%lcj*T!0=CN(C{e!(>tA(Jh#%<;>AYjThr?#AqNVm$_z<`W zN(78g6DuViUkhSra}^A1!?<2+x#e$f5<&e0J`-U7aK$!0w$7SGC%^95>A%vaFZQdP z|DMtJM4TR@?Yab5%*2mK3eihPoi5N^~ ztWXmO)}Ym8;cmoaRJXsa+h&_xXN^KeOwcz?$x{09*oKSUhq5LBmR*jHwS9AhtNT_7 z=KdQ3bgxbr2l`U=*)1$y%h}U6ZvN(P)a8_@TRS~`C+#t`i%gz2>ChlycW6{s3lq$J z+usqK&LRx=9@%W z4AA>#KLn}$dmVvd{}9Hy&LCSwLs<%s)JLzo#9cm1Y*(wC)y&hltCY)-xFBwJ=*QrG zXFUYaT`J_kd};98ww`TcmWS3{(iyw|uytfQaj28XbQvbxG21_aju*g20z^-AntkuA z&Rr@Np=T+NFR4U!(S|ZCS+%$ue%_Vg={q339Y%AUEG7=4(0Lw~z4D6V)}=uti4O;?Qb-VCP7QKP545`v(l zw;8_wxS{0*{%**qd8>ZRV@LuNr~pId0IX(=Bn?WfqU>8c`gVAdZ|Q75x7*wEeKuM+ za2ntJai65M6USfwx|FJ;oUxmpbPFHvMu7QsE0@C_It&L57dGF+s|FO$`cl67- z08TZc<-wi*@fV22Ic+`UfpJxUU6=DMO+iX*Y?6;jV(%ro-3ewoLS1$*rb5(Fe!@*G zTj&9WIpSXh$g{mA!HF=x(Oc(Pr!d7P+u(^}=>xk3D!4IWrhTGNM7nNBjh)*5al{#Mw}a$&%+F8##k;wJuZQSP`JV>(oz{e< zsUK|P2Qrz{0>T-b&%9+dXU5)~Y;MN`oH{um)@mUj7jwy3{LypGhA~9|-oAcQI6o0_RQg=q7bb=oe4nP(n73_ly>iKbvLWon4-rzN-{}s9wl`6@CSywcw+mRBg#Zi5ON4 zr~dX0wvAUG>uYvk{HuWxtrugFD((qfHjMe|WuEJ5oH85}z3YI7h)3JD6;v~raYgSW zzL_!JOWcL7b>Z@q(0N54w3UnU(eX7qv~flruIRU(H8JiW&C9!I5miffr(J@=i|Lwe zl&N)E>(fV%>s>dEH+0%!f#hWRD0Q1&AXH?ZwJc$m7IeOiRi;38Jy&!8IjBGE*Ml)X^U*?zg zHKt4WN7nCUYKVn-`ab0arX!znsJf4?#e`MB+&(s`^ky~gRTKqk z^9>GJKn*z$>1XKNX5D#B0r#fLmHI1uL5hwiK63ghmm|^%TX|`{>BO5)%Ql`rSj}1kMg2Def!>O=ym>)VyUxnwGA1^CQ{77m5O1SnrQj70 z+dSV2MX)6paa9Ep>jn^ZxVPKs5qnJ?k75j#D17Zw46gmn{ycQ=>We6naQ2M2dHe+C zo>#?zKe5^%7hs4gyR9x*b($NfIRJQyQIF1gg1$0-zDSd6c4+$vIl4TTO=DZZ^^Y;a zN~Sen?mb75BNASu^Lz2e3%n846qsJycN~u;7`%vY%IZ6aJY~VS>dH+SC&oZm=U@07 zJbi68%?KKJhYx#?n8ZlNGTXI~troW$XtmJOpnv)CgGEdcy)uMcv9&$Oxh5OOIW>;i zzcDp$q%3sA;n2JKDdw0O1`PbYL{>rQ?4zZ2N9bgEL~&JI25g`~^oMVbLfu_eIWrYf z`DwmG@~e=hY)LbBqZV?XhXqBdLP;8P-l*+LqU_4lzVK5S->j#h-nAaXP%mmK6;{=N zH*Y@bhk9y%|6JlV8J}|LZw=;;ty)$RiWx8t#n00@gWIy`$aBJoW@lOoiQ?i`&wAh| zstJ1;+rVk#dG}E?51Pnuz?%tGx41$>**remZwvJ#Gb%whloOu5%cr>k2`A{ThwaDv z*LUUA8g?gp?+@4bor}#eTbjuMvp3xq*U?B=j)u%gBeD&{qDvq|3kKkUcmDR=uyNQ$ zX0nR%HVI@wG)*IMre&*yp4=RM3Hg%TEnT=02wh4Xk1?G1JI^YN7R8BnyP#yZdE&-- zeD^r*xd3{Va$m1e`8dTzP@PagZfW|qThHbbTK?uMJj}b@W7GngPk5K@AFNj^maZ}Z z9Bn7QtprqFfX>-1L;vzgpMx_HYTcg05Lsn?qEQbqbeE5Bqw^fvr%q*j#^LoiY<$!~ z#>H}+=xG%{zS;B{ik}se0QTIx-W*tc>cgir?}L@9&JS921}JQQt-^1;?;46N^C<(d zo=MqXGqoOF25lwGf=s`aObsR)n}kuB9bW`fW z<-W~#OI+4RdW<`W)t>>k_df68ek%q_11!J&>OI3f#JKV>u%#xjJUCSWJpVYR|Jw`{ z7JC*(=X-iKxr%&uZMD4kIJ9dd$>%FXzOiXQ)Qjkd{T8cmz!M`#&9g`4+XLfK+S-~fm|;ZhV;)=8CtjtAj}G(EZXfX1o}t}hZ85I#+lk{mh&T=j z`i`X|KhU*%H^8L<2CW;&)+atq)*dW;k_f#5C;)Xa21Ll(P4AlA2kkY&2P%KC_Z{~- zErm!ZmC{ZvdiM8O{;Okzj zYi*Y^8=$&oFxmci@Po}~;9m{kxKXwC%o`I`Rht?Thz~PVYHQS;KT`ITueDpkb0XQ% zoP1p|pJ?T4{<+r&=JL5W=L|CI>3lweE4pUHZ>+?_J6cP38&4ujB}*vJpUIs5s$W86 zaZ~-Q>@o1shACphs0Z%}^{{>?FIuND(p+}~y?I9?$Xrez0_v-B?4M`eQG~{Cg)P_w zxftR4ndV%~4iYA{T~s%T%QMmT5j-mBP)<2<3%+B_7m%dUM|-*Wn!+ZdUw*pPqsNz$ zUp3TW0K)4(ODYa4*?z7l@V<(prIU_bo!d~AAoz01TM;s2&lIo#Ew3?;hG=woV=_e@ zPuAApEZ~v+odH^@d*yMzm5)0c9Q=&U+GheOFk`XNX5`yQ_O!!;ZeFG#NI*NGWVam9 zdH8n=ix4@yQ_|QsSR|@H$`Zc{xyWg~Gp5#fud6)w?KJ9DVFy^-=0WsM7w6L!|Z-Ao){ZdA@WCU+S=t3UI z9heDW^By4|4f;BQxSS*eN3w467<_{3@{&G)B+$S0eYwg<5?K(Ss-f9 zhXB-y?`K9lL6_0|KJi2DySn#nc$c=~u84mq9r|`&`-j-R zu-JU`wxi}Y$+m~9r`@zSn%VPCl@IX7~R1D38l-Yai|tgYrveIk3o9OAk?g zqJc*;DEK_K{))Gl*1Rc)ET_qFL_qp7&(~w>tQFWm(pDIXTQ2@M{K^G1m|l&i_6_ua zXp#)F?|<2YE}ae)yqg>tge+)LpRD-T>-ifg`f!^#ZvIl8r^Y54I%4t-v+Elor0jK% zTJ{ZXKee7CK_PsrcK1;$KYeD4SIbwee6HM93d;9=E*`FRjV{;DW_)*gx31cD=NBq& zhG~#An?YA%xWDRwI#v-}|n1NCS8< zm1vLhU1{~+NNApZ*uD?iuZg}Mg`8*uqj0CTeWxzYx?(fS?_})_%G!ImZcs^Kbk(;@ z_|T^u3=q9KF|+m|-t+b4pAjgsCv*u>E&GvMrQ+9iHgABXVkfN5z#IHK;jClO1>4Y# z@2RGd$K6@2?|F{^iB}?@g(VUUxuJ&-1}B)W6Eg!&LD$)C9Jjc$jpypZF;eI39T;fZ zXN%eE$3JA(qGi^8u|m8rnAO~L%9`N2L+NhUH-mJ$rbv`9+P@L?77gJ2MoiRgA0<0o zsgG*9@-O!$T1hir{|ve9O|?*F&v0knN{aVbVb7#BOkun!?PwxZWfUA=D!(yb`O`O7*6KhwXI!%jmCiR7V$Khl33lCGU$^b*jB&ou$xG;`Cm7ul4YC;i zfV<~9RAEq+-acWx-6y6=XM*of-S^`^GJfWLqnTuP&F#HboraKN_G{GQrM6JOi20ze z52QA>tmF*OjkcUejLw)vsOQ*24=W#zOS9!X*dVw%b>szGL1>J{bFFGIf;7jX&VOE- zWYP3677>ZgIy8IK^%QgO zW3QZ!nn0bXH;Cw}J>2_t#)}u$KlR(}&>+xOWWG&_DDRvKs*6yreX^zPv)q^!e&pbL z3F;=>zqzMtL&zEddSZu=4@*G96%S!j>Z^N477P!WIs$>Q;i%ZY%8_s4inA5?7UrUE zoMSz5Ua_NZ3$2)`m4aynnUuyJRfU}0%!Lhp!Vc&`cTmxp<=cB23a&$#SvoS%LtMf$ zM;(cOrvU}|6rPIT-N9ZuWUXdqE%>cqLFfrR9d1e~qz+#pBfUD#V5^8ro|YE}){(di zhF2o=CpqT8O7DGSi?Ig=$oi~y>iuXgN@&^ZsB<>j>E3Qodd_>JclPdLrkUzAIYmU! zeJov;Z^r>);Q8Db_G3jKWq6&Xz%D|G=aYmr^3aM9xYsZ07=Y4yAP^jVERF!yurN@ zsRj`p7lk9EHwaAbM|bt-eC;BAFIz9P&sy$_R+IQ|v3ACmD#84H-lO-;cwTp25p;EF zgK{VyF#sD~Gll>iN7E*kc?zpqL0RiLKJ*o1cALf3_UZ*}=nhgwCuX~AI8T78eeICo zvJm?wZMXObfP3ZY-_-o)lzr819L-035yfs7>bahi;vO8z$KP%wRLVNtHgVf%BRBZd zv-BnHY?gd-4AJc;>C1PrqHPjn{-?}NZ94E-$BAy>P|quPnAKApTjd)S&CuiUTHDMz zU*~?uo;9;ky(@b%_uU>EME*%SCT{I`@aPV!$8_O^q`H0gIQlg!bimjzd|y0=Sxw_Z zsY0gazKW**Mg!f1b{ttnH$*s+XnG9%tia4gszvLzPzFqzEBR$3t}Uz3jEVJGE2*f|EGC$> z!7}>=QAiV-WJjh6z$~W7(jztn!w8B z6HJ@eFZD8Od?N@%2R><)6}vEe-<63o@MVt?Un^u)etc*jmKUZ(G&fhdB)Z>>*E zXredTlgLig(e#))BSZfS#>#xsc%a2hUciYN5GhX74llUDQ_1l4-+&ggBY}YZUqxC_NcsU11n# z68^27)TsYiIPrrPMm8UqZu8tiVurGKl{8+)ho19zQR*XJ_|Aq$u;H$??EqqU@xIQ2 zHY(>sFe)x+gTCwNr|^czsZFXOFl70+9!PP#SkRd7ry@z)g(??bgxY!x#vtD+2+}~v zIW*|@spN?_%dwg4+Pch|CCavYRUPPc{q*)|TfdGfw%Vpy?&Hv?u6Pr(yvY()pD|+f z4H9y)2wr-U#{aZF)!KYymsj*!WFNDYaUIF-v4EfvxVW-rn?DeRZ|5jho7{RG2~}bG z4DT}Z3-j4AUij8B52kV1*QI8e(jnyTCH#CIyxa)Eqz^S$?yEJ-qK(4l@pR;CnDdZ_ z8~X)*>9LT%WhV{{753p`OTUj#UD5v&w2yoIxjVmfH;v3&l-jo%tbThYe-^1b?TyD~ z)BT9~PW)YH#$}gu(uZ0HuIGm~4ZqG5PXq)5=5(QcVg+J;hSOn!Yl(M*_;2X5L;EPH zsN_kn^xr0*kw)DThw4OSXzI0Szi1`({Pe|aD%8%*MUz*@kI=j)y&^-{hzMVH3HR@2 z=J7#Sdq$A{gBfjJC3~gw6Ofq)WHf&&|1uK z;784ILaBv!@lV+U5`yT@ANS|!q ze8X~4ReY*YI*RonUXO)%S9CGEgNxaJ=J6%TZ>1qz(veB!n>@g6wp;3gvY81s zJ4@kfEAZ5>um;xOS@?NdvS&&_;>L6Nw)A*4`!};bKBld8)NSvIJPdeYl|mG{jGu=- zX5o9s^1;-=yA$!o7TPiV#65XWAHJ+PX>3^JRimjH3A<*ge}8(XaofvuXPHvsi3Z#; z9Ru!sDQEhv@RUo2L^t<~N|VYUJ$&b5J+US0QS_5)plf_;_HL8U239jR>}Rv!-~i9a zM2Xa$eB#U?5dk?ez&M%689appK1(;iu@y*9m#XlLlz`ht%&VA*2$w8rf&yut2Zwg) z%_K>!t9}{E&T06A$8VLXg9ZrytCh>q&HnNx9til@^s?E-VAl34BlC}I1+#iCH7m1QcLC>v7K1sbeK*uC zhLUhWZ2v;{mE?&wwv=rjr(fpXS`PPh%Q&I<_K>c$3c>4#}3-}$=a=-lq!9?_S z{)}$ai?wqB&oaMnfcR}IkpXN5byQ7!tRDuM%+k;!K>g@%{10^;IR-MiTHCkR6XW&~ zJ?Kj9$>yMJBSoz~1VX&dU2Kjg%7xKx+uQMdk;R%+cWp-!$S1;R=AVsL$we8In-?P6 z|BYmVHDd`|fj@AfrA*zOIU9ZXC~JaocwUhb4mGy6^?UvLla1?~?h&ClZ|*;0+Cj7n zRUASC6itAYo~RNZ9ow&_PxCix%qUVt?aIqSvGFX*yGn?YkVfHw3mY??ilh!D!_st@ zxdjQ3NSLVoj_EVO^=TS+A^lb^(4^W*Z)r7-lCH`U7x%(pQ1<+ zZR|j~Oh-KbQ{`T7MO3FS>bVvwZx_qlZ6n)`r=0&3K}?x*s}Puk&G?r@)-!7*OD z1cdJ5Q%C(L1Rj@S>IxNKoYMZ(L5Zwx-cozjli(+6h!GdUmQ2H!38-HWiO|3vxkRYP zZ==59e4cTS*uiuwyBZ8gUR?LCEeDt5__q=Gg>#4hLsH^Vb`cf`wj#NYF(ir`yRcH<+ypEC+*_uGi)fySq%{?E48>ZJZ(@u8M#whzO_?W;a66( z#yu8fbMeR;wC(pUEeX$Aa(4nV+X6z)CL4^Zc?abzTFkpFev+!tIrqYoO21v%X`6aX zK)y9T(m}QDFBXXMn|G+sYlqikshLK2QfAP)@8I5W^;`sR_WV0OG+tw4`rKDdlw);* zEm=rk51<-eWJEsnZDw~m;yD0@*;83rPcfFl4?dm_=Jsh6#_CY$bfA8xgdsIF>JjO& zapUZy6=}=|(8Cx?H%n=lINq+L6{(opq`KP_F?5Ujc)m_1aUrz*b7r|Uwj&pBk)A?p zJ#^(~3_qAo)jtq15;B2(>hKO@bHCB!8{`^egVK$X6 z=!@5H8b2Dp>NlJakXZ3ZSv|vv1N}d>jGF0~>)%>-{}A|-l(Y)R-17v}jxFXD>0sk; z)1saib97*V(!h6RDir}uxq*o@zy5aU29SNVU{qtrb9^?W!2dhgwbX&`=CB>cI7mnh zYOu-$@jThn;q?oJ#g|d8k-SUPNG>*(;rOoN_sBdMR7xNqt%Mr6P#-&HWo$;YL0PvN zQYI_@XmRUEsvQ!CZ@-$A<6JwK!srn9oOy91dpep$wvE;s5@yoH|BGmKP5;L`fJs)# zdxJ&%Ua1Z}MD5Anjm!-%g34{d6OElnIzi*Aw*=z_+H2N9$&!OJ%W~g70z<}_vHfC` zi_Q;|Ozv;QD&XWCHKlV?QIwA~jAG-|E0((6h`|F3`8vXJnP3~c1wR%eNK%?#M(Ps@ zU9&@TW0`N?2WSQ3?@mPN@CGU?JVp!0aT#K^xnGW@X;$c z2>)^Xs%Q>tVY=Qwl}s(^E#Jh%VNqIe(i(#e0^y#9S|9#s29G;Ud4Sb8^Q1)SEEM>c zh_?^|yza)%HNJXrXB%FQ)Q6s}s4LD*1O`!z@^-(w#x+;L+s^0Drz@=~sX{4IY=b^m z(osf7&GH=38n(k?=cVH|&%>gFafwRF(APB@qu-X2tcB{xR7&E*Ld>6sePVo@H0iZ7X^E8*-4cbe7~&kB5$g(|-&THhB2W|I{o1#&>}Ae?MNsCVyJiWc0dsh*}><;`G{o$Le+FXJTzg z*jqbGLB9*`m$||T&%|9b$p5yJh?FXP>6K2WDbz*$rfH zZ*SmrSwu5pa~KKO%k9v3sEx(h{{1{9p*mMi>afFs{X?;14^3P>T0Hv2Kimc_D-A|V zY6WFGN@RhmJ^NukYX_0;8ily#$QWD?W^kLv6b9aGr$5CzbtPqm&e(6fNEVtV;qCcp zp*SURQ+T@LxUbaljV)ZLM3Phag!EZN@BBCU^6>YPmGXhFD0cD0t!po}j=09GS_x(6 zu#c+*BRK7QKHSz0@b-)~^vL!t!c7?AS>T(Je8=^43Ri(+lhBu)G`81V1hu_zp*}VP zq&V*W)R?|aP%+CEdJckpD<ItEa3C0a#f}%pT{w z)#ngZniqP#;2BX56z1hfwo{56bt1Q{e$G1TZKYs0 zp*owZ+&2C6j5j1Uemgrtl2mZvdzxwJ>cHJwN6# zz(XBVlNh?Z#m!>YzJ>Dq4!_|?KPSnVh6%H+`H+l}2iDaJDgL8O3_PKvST8vvsNtM1 zgg7EQ%j{@d(mb{#X_Y!!XLesqnu(%s{7LbMN^u;Rs`%z*2}pDyJi9tiab_cJES6sR`Zp zrNsoI_0K*^^mx#I0~g?B`@fRPNOlb>@lRUW2N?rQ|4pljz)uNM&_5H!?D{`GF=uH+ z7+O?Xux)$36j&;mkW{8}s>D+8(f5J>dSyWxTE+RMsE4>;sygGXuuw1ELG`YmW>0aT zAiLA=`61c_6UWckF}f9gC7LQiEhU{A_bn`p*4=+-sE6=h7CzfuxIPS4eP><7puoT$ z@jb%n`+GB2rH}g^HDphEc))D-(5}_-*pt6+j89xExY#XzQFs@^~CI>Lr=8bGfaAnEnT`)gUKWd(!YU{5vgQEmS){l zkSAJ~j#E!$qJO9rEMD2+va%D=A{-(X_XI0F#XbSXLWZ>#C!)ym2pYin7zh%_C{a2g zm(9k}W}NzlZcj0l9+Vkbii)Mz-dYssOOfC~yAd<-0X%^y$7xN{dcjm zt3Ovo9!FP4BT8$(6;;-?YqlNdf#a!}@%Mbeo7jU^NLG?bMS{<<%Q6RW=1bL2d!dXe z^htV4Y$ZflBFsfZ*|N{lhQC@ZGraZQkdtbN22ptY*rmcb`$Kt?dZavw#^m0*&@e6f z&RBVxva%L?3^w4SWF--(q|-a!h9ER`0=egc=&mE=;F8B zu3$M)y81DdCYI(SA&-j{iE@MU4rE1+^S9Lt>ioCI{CkaI0%xzYBXSSc@Oj;67(AT> zejek`Mm@qPEq0L}<_1mC`J;(GvT@KhmX9&fGwqCbJG0S~dN4U_S+Ju#zNY)&A7^V= z=2=d@zhtsQZa2co#9$y1_*~?qis#?H?my=!Hagx)c{-WSq1=DyDgQjZI~dvn$w?vi z1BIV#dH;+A65Xqm^&pF^pZ2+M{2B*0-9nm5ViTfJ#<5Oyj78t)hv@Fc!Dy^QkPB~H zK8-jkR&CtEwv#W1<~C19K;{4-Y>SoE-Cw`pXY$8K#4wg1TV*7$Gyir;|J{neS-$|Hv zm!~CV*Qpc#9*Mzr^UN|R;GO>@mkus8*_)y-P#DJMdYjvmln?(Z5c>E_NHkm*gBM?j z{#6Y;O3KIE@F;Rmt1cpr?`+V9*CC|Szhb^Pe1sC45>)rII)!LL+`ja%Z!fCx)MevV znzZ(g_?WfzLCkQvZkkX(Mm-Bu1)tj&Zm3S%Eg9x1sx%aa>!j`-g_Cz2W$g3q7H^s* zX1cU)xu4C;$g-$P6^RJ#LahyFRpjkLxw-bPJi7-75-ymTSY5&uzvPMcaLlVbs8)o; zFp-__bE|n8QE9!JVPqkQF#c84q(qCg&yACAvvoy3CV;h^MAkgKL(=ngTkft%4wF07 z>9#8`n;tM&x<^rISj8h{ls4*`Ro1riL$|-HiY$B2 zUL_!W;|Tr6Su=c{WkNf;(N&9ZVedgA*4%=*-(OP4g*H-Sn1??Px=z?bU{LN2D@kIw zZ)dElGqR4!WG>IqtkJzCrOAS_69>!COked#=(1|XCOazXi^&!iHd{)u?VH=t>SwAw zX)1V?cej9Ews*2R<|+=`(h|#?%PpSe+vOSaOi$g&6?&*EfME1|iyYt~_W2PRneXWx ztK>3B?PgouYuTRzB##8WQ$Q6J1x2`UcI$B+1jgK&&uA}3icr)?+0_+5{lC`5Jhs4( zccuS75Ey=U-8)x2q+aj;Mfsx$(a+_az0sBwm@g>af@zJDcjnmF_Ru!3H&N?o;R4^t z6l234p?$tt+UwXgob2a>y-OfTFg7O`i^@Z+g9A&Ac$b+TA#BFQwaqQFo#?X2Aw}s2r4w0z3Wo3lN<;8MPDCV8#$U?FB zH{6w={wBRd)vRwlKV}j*m;dDbrF0emf-KL-)UK0VXPeaz*h|43i_M!94U!x zMMv>@>yQ*08q*Tf+=kq5=RSxKpuKSUBntGI?3~D~>h!T&v{GMl1D`E<-OHtb|VaYzMuY?;N=7WxLFqSwC{q`d|nB z&EIDm;Rfmd^(O%qKr=!yHkLTkFkfAs>hQvY(99_G`ZMT=r0LP{s7)ymsbm43HNUo1 zfsdSK1FtE}yxMR>tf^Jw&$ClRGn0^cCF*7n^Vs++Pgm4p7He@tp9>Ge=MTcZ1>;g{ zHeo58Rmmh71*ZwiuTTMGi`g|btZ_Q|&QE1`e$n&`F`%aaLh%+|iAbRf92A29MCxfI z$zD!hw|4(83_Abi*24~hUJ)gcFDkZfRte%*8x_-xtj)C^K%(ghUFrD2CoCtE`11S1 zAf2&8L6NVNs!aUK7Hw*>FT}{nc9578YuT+6URP#H8iPJ>v3Qj}^-4H!;CJa8#>G-+ zzOXv{{DjRXc;?aGWjI3);Z73dYQKP2h}$s$(>WsaJu4HL5Z#)7X4IovZ@)PZQ41<7 z!C?$mD{$4Kp+kKN8D%Y-NIxl-2ziM{)b^UamaI1n~WT zdbyuJ#kL|MsLS5DOLzWg7HjVbRjH`yy`Z>d6fHIVByV^d=ah2V_Yp5ad7xu|H3PPR zb$7neNH)=uEGt#7<-`^+&W`;#L`EFbf|g9NKO`vUOK)35&#-pY4|2u+Vw22EWzRhv zGLNK1%xS-()p!#3mdL&ewdm_bkEr`dR`t3_f<4BzP;#NZjg91w%?Y2E*{Yya;qGC@$rML+@-#dJ!O7|*jZh6 zZ=(=g_~nvEYnRT3Qs~ z3vNMsg25CW<@NsGf7V|ga=eT~+xABssH;D>Ia>*2W|ICHc@emiZ7JhNMj;k;U*<>Xz2!3E1g6^ME zV}^?_NBSpZ_Es7=OP5b!wM3#84hdQe6cHE@*Q;7&6YgBu?wST z;6JxtcvMih(mXI!$6Cp79@BDH(`=Ypx?{J0jr-FPzEOGu#G+}zO_Mv2?0N3qo{rD} zl{7Zh2`x>SM9FGQvQg?Bc?eE2LA}6<`iL4htK+D9z~9@;yGr@pX_*V%0j5W~vb98V zK0}6HG;b$yF89T!k7F1#Fb&w#c2m_$pI<7p4$-28)G0DDSi&hXBct!1m&rIO^BxWR zM|wkI!Pc##V~KND#DgMAI1+74X|eP+(Ph_Tm+Eh4pATnQs+K~cszOYqc-AFhOh5l5 zh*}S`Q&*Wo*?bE;wy!eiU;cCz4UQ6{;|A{M{DVceg4E6>!vX2UK7VTT!*aMLTviT< z@426R{7^&kIX2xr4a+(Ca3g=yN)pI!%(%y;rEC*a=Bvm1beleZc7Ig4Q&+vbQc;LC ziVd5rhL~C2jhX=b5x`ZQg(p!+1*v^%s#2m8^}T7Og&yE~`8m~xt#2`c z4{^5kzLJr10|XQObotD7z!qN>0bfgjUkYZDgl z`sA)`e}XqRLb1j^pSX+F5M_rXKf-d+=v+O`3T}MxivLd`76AdJ<=T zr_oXM?Z{lk!vVHW3?XD=0NezN;vSH*WGD&yRiMbdu6UV--R#H?$F>I=}AuDSy2kU zBu?Uc)OOOQViC_a`EmCr^y|E$cHvD)v5K9xtYBx=CrRS~RaoBDhoznHDkrqP6iX#P z3iBcrU?g54dNti%6)v!kiuMf$6EkicZkI>qKrI~l(V^5Hla;yva_O5=H>Dl#o43uU3$ zqO;uoLXDMn#Vaq+7^AfRwO@7ZzG5*Nm3~rIgt%i^ul>$ohmDdN+}h+wqN4V?8vo?H z&5}Yp{yo!-ncK_<#7?BN$7Fz*dyg;HfhTivoDdZ^E1fDGrAGB1Lv0ap3MsZJ@?W|N zMa7={1@w>F$vwW@p|;HZf#}?vvxg2eWuFKQjK>a>PJR-y}pS-W!3?zDDt!)^l`81xfx;AR;HE<>gx%X6# zb!>Fp1ss_;`?co_I6n4*CjT_rc+4|ZgWbx8ohH^n`Gvk@G2zod;|PjrnZNUVTLr_y zc3g+l>yvhYEmi+3Jt`O+R_Gn_h_WF~`(=pt>!Q*B>$*hql*4V>JnC%VQW^Yr*b2F?ruCQN8FG^ zmwC&EBCckvgjUUygt?kRB1|H?DIHP)0gM%$xea}R_EcqtuueTj%5kvjS4Jv6tf=h> z^>@ErCP-GMDgLIF-+BtBPZoOYmqSe2u%E550S@_K5;Vk5z9gpb!Ar_*x6gY;0x3;l z=-FHPYv0? zQ|^;1XOd;33+2aqJ?EV-qgN`3cZj@U#UJ9;tB$$?GQ0YSkM4EFWIX&@l@gm4KW&|p?{d{}TJ}o~9hCP)6SVKm zN+l~LG|@$8j(iXuH&0Mu!s7m3jc^3MSXxUhtL z@e?kF*q&shoC;4zd;wv#fBb4UEB`6q@WJdru(0lCj{1vNoeC}vX>(zk^R5}{pL7(F zD$Beha;;s^$V@5q(*P`3~T6qfkx0uEfH~2ESj%P*a_Fb%?hPCjJ zcrP(8{xiILWt09K#v+^!J_FXd(#e9CN9;8{;oZD!JwIEQ{Zm6yPCo$O)~w&X)idZ! zcOT%t&Vof!qQ?J9U-huptN>B9w4NmOJ-jX$(l>%Si*luoY;E?V0%#L#d@3`f)stpY;R7CY0R%%PRUWVyA{aTEpcDhPUR|{!i{X z7{2)fXk=_Gd7U%z4xLT8pi`Q#u7O_HMzfp2Bo0+6s@`NwlkAuHJTDs(8O%Z!Wl_}g zL)nKC$jUG`J^qVsGv{xo#zK=&%ronWZk%`7tDV1om!+i%8v_ha0-ro29|K^iV}SUZ zVzhO7R@T$f9xl6v9QVvY9pWh5W|ENiyYf+1MxR@@HWe>1&gLI?B|8Z&*j84kvx1y2 zjcHPrp!CcvS6KETfv;M${y(zbGAPcjYZeZ{gN5K8f)m^a5AGh^onXP;-3AEmZh_$L zI=H*b00Ru}o{#%|p8I?2oKv-{_S9T8HNWAle~QLq$D$%aszlg(gFg!O*9;jfmhOU?4l()iVI< zelYPn>(Umyy&K|dp+)ymf8HMC8X$d86B#yeJO?$eesM>PBaQu_X7>^9@sQ;uGw}zi zNLiRCM9&o{_^j!}uFsN48cbmi=ITWp+!?z{`qr|9F(Q5foClDPzVhX$_!1DWRW(iX1c( z{0NUak;uk)X|Kd8@D==Aac=GoZPNFcc0zLbvmLa5k^egBL6LXrw_GX3D?Fm)J9fm* zkA9MU*Un^;Vf36x`V;o%3jV`|R%^+xd4IIou{#hM)>7R7(jbF#ts0LV?@^io?5B>Y z-9isgut$)v`@$x!Jf5gz_C)_3%$Xz@Zv@lK!5$x%^!~40*~dZg6RL$dx6XW(#?b7m zYXcH0b5}gc-21LEamEzwFc-=V3}1Q2g$1pLa-QfxqzIz%NJ2MsRC$Qi55GqI$I@-T z9Z~(Aoafd2PKsL21`r}T{iM>}53vbv6#4G{aVIMQAVpoLfd>W4n)9YFr0aEkXFTPW zXy27T3F&eCBSXdoHqY*2dpsaxH!u41R|0>@1%qoyfG<4#3YgBGgpuX*M`EQE>{mrn zMesHv+19vC(mQJ^4+DYGtxxAe3qd;Y4V@*K`zcVh}QX#x939hFRAyR9YXbw!-ofe!As@e z2L?k(2eYfI^U=<?dcXCQl)9zh_oHI(9#Gn z?GLfeJy_jpuNl2L28$37>CX>+ksOV$av#${`8iX(1RrI5YvQK_6jqD+TD0ZD0ZrkW zll7A|i^*&buaC@7+^OpA*JJ$_++=k}CU4K?4l`_pqoDnqf7Xk`6SE0N!easHj`D)^ zw(SH}!x3bbdA;|k{Cvgzv zkjLg>Hgf#5#zz4WRJ{21Zn78!R!+Q}APZS+jBTj$+KD0|sv*hOdxc{C)$^|Hz) z7d+J|!vOO`oFr#lTIopKCR9QQW28K5S6i@1p48&`I@6e`uZP%{$B>Oa_xbPK#jdE? zrNKosmY8z`fg%LXSHT$9MT+(-sf1`EdZ^#>uX&#wYtz~6vu~fAi|d0%IMfIIbmlG| zwZ5Sm;^Dt3Vm=nx#h3B|AVbkTe8Ko61CB63Xaq6uAsr^6GyEe(esgmiDI*a)XCCcK zYelsQ@fe6;G`r_tE&|wHfDj;vGZceR`7PY>qQB&i zcjP-4#;w$|fy|wrP(knX1V`-0R9f3ZZNaE?n2uVEr0k?HvSq1;AEi!IVIlRey%Y^O z`3u)c7p4{?E*z{Pveci%%SvI^Dby}{Xs8tOoK<|deK}~j=E%j#8MA-dXdOxqyF3`; zKdKSdGm=s6UD0k_sNU8j5a}HV%lH_vR~SM&mEYOSD}qSen=WidJpsJ^88qZ0ALcCJ zmavE=%Q@&xEA-e{7hNwH2~)i!P-OAd8}M250D`djGGgSP@xx-H zz37E%q@|*Y_ln@}TdQd^z5&)F?{m6@X5z!ewyM%TOMm!bp^vFYmucbahxA{Me| z0H!MIP<<$bU^Vk@px%;=A@osR??BomU!^AO9+|ov&QunaAXak<^QrT7)SF_{DWNr5 zULYN(NH_D_e4$X9zv6eBz;tb1`^aDHpg+6jtvzBWIZGHxV^_#Dj^EHECt-GSogt zO2mH?etd&4&FaZOR~9^GV*VCr=P$JYB1M<6O)6DK2R@vj$?PZr5U@yA$}Jrh#rl0UQ%27VAe6qHP=57?Yo+8?Xt& z*ak!eZg>74wF%{a)Fw_+FRp*j*PGhpKB6R^V*|;Don-kL|1OH<2H9n#gHrW3wIpG} zJErFzd#3K_N$wkD$*jUr`#=?aPu*%Ggz0;i0x^fp-ByZX%+vxkrJv{HdR-E9o}Snz5Yx;4YKGORJhum&}RT}u+D z3B_QsvNm;+UR&m61 zxKYrFaw4E_>GmuYrM{8Qoj`k3Z<0kb-9Kva3_5Y zvw45s^n5d)on~PjAymKs`tY;y;mCt`AId>WxD}ZJQw6ubBwg6{XU$#6sS}h<^Fv)7 zwxoLX8*AyCjY2AK``#6@Rr$C(n_V{ZHJIPViW$*EHOAeXwy)mWKz#E@JQLAoNi{UrV9BPdDXM)u-3wwT_)>m~dqfkw* zTj_E_lm6oSvm^T{*1a}eHGqAMTu8&@_sq)% z#3OWb`zVu?@ID&F*2xQ$z(GX|Ik0dun<)Hwv5-=?g{*8Rc35Z`MyJI_GSO=#zvsaM z42fN*lwXZoB$2A(_KC@2aP&*}B+kq2L`|s#%<)XOI&vg(Vk9CxvV#ru&DqP5Cr|eg zLP8N+Jq0QtaoJz7ledpuC)`kzIrlFT0f}KJ!IB*cu@JIriRR#UMQF0qo_3=-NkZX3SUGiEPT3uiE5cC*i_F~W=pq3d)~fp z+J2kO`qymepPlFI;}W;TWE=T1*2;e$-1kbp%h?Bce#8CL?Cf9k>Dl7w1Kv*Yd-BB% zN)*NmnOJ();B;=ULiFrTpqg(tjctpNUUt=FmLd-T?xse#e;|!6A{e*@j^0!Pw5)!xa6sT8i?M6h&N*3*A%6Jm` z%ZX6Bs@8dyYs~DjZ-52c3ypPt?va-vhap5=!4$WhwoA}>mjfAf({c>B3`UU3X? z>F*F1jjTF+wHX3UUR)R(@SaWhZAmh;YsO^H@nZmcaeM&Yf=iK3Q3F@ebGEDw%A>!3 z@bEv`!$regHJWl7Bs6=g@6=`x7n8I{(F@KsUdcaDGEYrNzSot)tXfxF0`U6SzXX31 z5CZu#zfRgmAxyH8#rWN0+%6u@gnMK>KCtYVWxEL`Ng3i%F?&2Y99~%1t^NXH8_4gW zg{st3#4i3kwhDC3!awmouqaJhWRE-Y-?`zmUP(CR#A>{oGGvyzM&7M55I1JM-};?xfF+ zmhe03*g4mDw`hvNt2$P1!A0>9wV!X!#0X@{ntOn34?J%<^1JJ67TQQV!Z|qs;DKr@ zJaYL2`i%#_4jac0ZQ*Hj-c3jMk6k-(lVd`XVhb!&&{$N0uFDEiJt&02HmE0ufa9}KsiCeRB#+{dUj`6WBYhYFh@%@I0!HXaaZ@T2rC$-Yc7m6qC zxvt@1-Q&%x@yAHj=H0?lE^)fj#oFOagQs>c|A5xpAC;89b{oi^o|NrQ{ zu?QxL{|dod(}C~&sw}U?WjQrXw?qQfDMZ3PEIPDLVDmAmd*n&{&<)-eL7~MHYd_#B z#mkrq&BB$7lsLN|Zlidn8b~Dm*}}?@aKp*`arU=C{Eq~lNov< z7R{|0D9Xnr9+8nZPw#Xml<@N1>>pX3Mb7&~AHkZlaq|8z6XDwlVRo@-VnBPw!mLPc zV^)lW!XvoAE3t$~>}J}e*Hz;a8WN_0+)>NOrXH{4QN2}Jq6)`32dAC9Kzr%31cC4K z25d8bhFTs9Jak543@$Wy7Wrv51W2c$4p(3DXMbPhoq4P=&NVoh+NphAp|7rc2gk;A zs%)?Go+xDYlMNtPc^fj|tc`cc%5Wr~J^X}i5U`vrdnvCXK5N>me=fm4PQ<$<0QOjM z&*3$&-sqbEC&6_udP{l{Y7U$}1mc<&Y}Qb{uP=9<&W@E*MX5M9QLa>8o8lRHH=#zp z;G0i+pD=RN(m0%f3Ff*5KPbKM3|{5=$vQi!)DH$_R~aF5F(u%4t~U4>D(r)g*mFO{9^aJK9mdQZmZo$!^)*$R&9- zC43?kW*HiN_pL@8cSz;VUqDVbT+`G*Et+M0s#;i5F9j*)9Z@7|MEAGsJ^S3C^f7ta zwcs*Zax_Yv+pln@n#RAs$K#sTo=mJJs43aH87d{{h%}Z)TZAlET@Qx~%|=qoJGO4B zz?b}tNWzAG*^pp){wBPP3{C}(fjyr4OM`C8)wkJe(b- z-&P|$zHLd8Lcf=yHrY@nuakQFQ;B5Y%t5Pw=>w@@`TY)z>Neo5fc@3$*eqS!3J~Nd zxqMY=AN!<&H`!F5c?Q|*w~P@Ih-xFo{@eQaZcP%LcV9UH z#8IjOt9=yAJew%Mp+Zs2o_ZIuL+?v;obY$IaG~ z{_aTeIKk25mp7B@&pzYF0@4-1KjJ@hR-m!M)e0g4^J~dutfkg1rN*nfMaqC^%E1gWS_cV}Oy!&v0L$I-ves}R8|w**Hw z&bS7S1f;>EjOK9GE+0Qloqar+KEtLz{pZ44<r#xjainV{wDTQ2PHg8C3KaDS(2RBE+i|A^nI5af$XXsE&wfbRFlUDaW9jYK^qZmBX{k_BZ-r}>M1)SwTun+#u*KDPY9 zPUMCkA>s+?Ol({_PehdB$&Y%JSqM1Kb@k34 zVX7@HuUI-v7ZG})_@32zV#$*^4r!f>U8?7%9j%&S_S<2Nrl6KN2;f87DQKT=t}Xn| zfH=8!zUSZ)(eLT3+?WgcA^&F(8eS~HggczbREqN>Nkyr$IfIgN0H4lWIb%gkKlJ7} zFOyFLDbZ(|^TVDJNF}cC!McW>WdzcU9Y7( zZ3j2YeT=9N*t#k zp=RHA{r`k|n;(}HqqFTPY9V$1nhG5V3+McV%)kmAPJD;=rO(7hVa~l;{9#2$4P_O7 zD1&0Fw{(6c)u8O(+8MrnDCR|`CO~V-85`^!7Zrk$mSi{x`c(QneysF^dh?%a)1rqC z%fy$&Z@lf_5$pt&_px9LCT}7hh(lJgX@}xVKwpswI_+$G6BHM$++{c17kRKZUZnrB zZ;u=D3-q?c53raEbLG!@JN!j7R-eM*4Jv*M;XUakd`r>XMvFmqo6vX(E#JYL5Lo0# ze<9L#(6Qp!ZgvApQHdyNpbQc+`<<|x*em*LRw)YG?dj(3W|oG?JOOid16=!2yC)5NVKFJUivMLMUeHFNSlG=zb>8pu#yJ>s$Q~7*p z#H{*ICN1H+y;t>}(u5FN7JqJJn}gB*T?Ic3WvR8w!B}mwLg{>dA9*?x8-BjH?6l_B z;{C_FJ_zi^KEFn8_GvjWU{5PhqqIe?`eYSD(@3$Yjd6D}Yp(eM>hzYg!RWlFX2rR| zB{bS_hDPsZQQW_`ln*ih%K8KAm$8-FPgCJ347&15`D~pir?7vD1;G%NzYoRE9Hq9g zQo%(&OITyxle<-7)ZmJ&V9Vi?>1a+^KCXhR3)EY+ozH^T!YQ#TkD#!RCgG0+(UUss z6p`aCOeLj!?^NkFLqZUYH_8lkXZbqW&S!WfY&)_zNo3wm%`v{qbGfC#mH)~ zBELk8?O{@<&bE3v@o>!bqWd{IQPpB~fN$|+Dx!xW&dHq%PS>TODK9>aZ&S0X&@dY- zo{%U2eQcmgAsc0yR$!e5`pi7`6tN38Z@Rh=2hgU>oo&8*0^lFquvcMtOx!(N#a?O@ z2#DAF-IS;iEP{EyKXww67V1x3zepDoC+2!iAAibNkDns;)c{NPl}#Bl|I0D_XNR!= zLw-N-^Lp9+`@kR-X|5PG1-;o6^X;DWw}k~x8;Yu1NK2(vm?jRtIxAre&95AwzTA;~ zW3nlF#S8w%cCf=N{h^1x@S>6mgJ6+L!@mR>1@q)d6;Z46Y}*7yz6EzTwHBonTUu&B zlhcdMJ>=Orv5@5ZGKzPDDU>JFi^~APcr2!ZYmmQwp$)gbn`ho+g9W~be~CI@Dz?)W z?EkoRZE8KOZ${arfU)8_l`T$i-^LE(L6qLAIO+tI+Nrd9Yy^Mp7!bmD`7!O}LYOI0 zHshkLKD`L~T0?THm%)J6yBc(_dTPl<|F)I5LwN?Xjj5iV4kl)xm1&e&pScW2@k?-1 zLqdVe!cTh01VN)UB*rovqOKp*(X=!)DY*f7oyd9! z2B@5X6}?iXh%lmmqpNuvV$a)!RN@h11|8``R6kkvQRhpa>m8JkEt1(sCX=49YaVFf z42o>9jP#9AaKV8-fBP&dGx|PpQz81zG z&iCKkbZh8TR{hj{_SrSxuYySpk}n;6Ld3H8&#euh8pP{hG+ zD#fjS4drsY0Pcv3qcUHK{TiATFpB0EF~!jbQFJA+1yGR6^~NMwypF^-to}{1;Bm2a z9uOAcrJ2ovQr*YB@S*C}JSXve6~+s|q-x%UD-uj#k3r4}M~mNNOz#_JTKm<@C*s>= zd6^eHBa%&8!}C4+wB~ob=D5kez;>`V_$Km>COB%*+Rt%9r-k|t(qA6OwX~|M8egdJ zX5^LDhttA+rzCF=J@GC4&tb55R}aojvX^7%Zc0@JTRWML_s~{uh^j}r5V7*zhnXm$ zJyLGUt2W-3HsQQ8gwE4P`jacGmm#`$pq~Xlv>gKUAKJtnSc5wzA^Z0}f__y)qZE{} zNOAlxjdb44)Pisk(a<|U)pFV&(!uB!U@k;|IVwMvi%=drr&%E51rRe#Q)EEWD)2VG zJi<+U?*gX*Ll$R%5qe*9`V3;b{Mi?sQ7d^lvkS|7r7$lp3@7%VwdExz{1$IBh?RT( z<@5Wu#V!1PZK}p$AQYMF(<4)GY>HPfZ94kA4VL*O-A_SGQ=+fb(6<1VV8c4lN{bbH zk{#JgJ@@xxy?%s*B=^tyc7#Qyz>SlKp9RGb`HX{-Y8dy8Y(Kjr@$o^T4|W-(`N{8S z({{H=9BRrq1ZH4$3Vzv5g;Lv0B+An7)3t=?U);KOA=kifbaL8(4iT@C!dCMFyd2~x zcTzvx7$?}tx65;LPk#(JwNW~yF<$6ime~-rD{uQo-zxXgVrqX5bdhng#+aOwV&^D+ zs@YsjD>J6|xc3B6vo7v+|KUQf0h;(c6;tecbCDUn+0;Z2CfJ9t%Z+hE)?pPVk_vE? zK<0)G->@EPVR3}0UzF2;LQNb!mdGAYsJx{OlIqFm$m_D-^|UCCWdG{r`{M$sN759dOY7yM18PP{A$a# zKwySc(?ZWquXmC-PH761Q+v_dC;SY%gQ-i0ql%~*Ib9QstUTGKaw8RgX1uW=fZty~ zh0vPGeI4c_eV2~Km$Zx+u_m`wD~m_{WaJ!1BgVaykiR(IX;Od5Y4V7?>{FqiZj&2} zKF*;p@7#M~ypcwmjz8Ui1k%SKUtVpqeqJ}=VYNsg?_A^0$TW^0EigypAngJ)JE8j!IIQeK>P6WqqTR>V9{iSCx`V z>qsXEwuwGP=;CNn<*4B;&F`z?Hi`Xvg4W3wJ)WIDuE*=d)Vz$(m=Nr(6^(98D%%pv zsT(HXC?ZQ=jQWxnne>*w!BL|X4xPx|4?UdD3S*D@b(QnWvl*}H)^{pD;PDG9Jo1}Sn;JnSK>>u3kmNsZYeh!4k`Lxzf7C!8pIc;OL zI1BXq1D}Yl#4vz+wuhTfG_Ro^$MLNQ;8m+0=9M8ALYMr(4@f0KL>U9%GqUh<(SJ{r zFkB=1`y2fft5z{@(XHCb6t=%!oCr` zI(Q64Lp8!x8O;&Glja0C{*D5C3vz-Xq~F-Th#^jnInS3NFT(P#f216 z{}8yWgo?qt@F@pZtbBTEMQ3MPSh*x46Y>tydHsxiEnlb$B6_I~B6u`1<`z#Y$f#2g z;u+XBsg}Y045kSK^0s-%$XZA7-moF8fXGTq^hQp#H!Qq!{*Mr)^cx7`J%3~AdFYHw8f=gtlsWH26rS~ zDWE4zgBA~Yi_V^`ujYgazJ`bf)dV9px{ic7_(j~h#A!&Tcazl7eL!7U{R{V-S7~@M z(Urnu4f6X7vg;zD#ezj%vkd6BmPxI5L9ou}gEuK-C2ob<(%Rw%E4?~A4L_$Nn?;wu z_r~F5{kJ^56)D3P%qgq_sJ%jBJW{wnQh2wkf`5(_gdMe!&OZjvTZNy?5R~W8MtLiD z30~D2AO&3qZ7xwB=2-kz+*+j>e~^BN>r6AwsT>TI=cjAE#46eUm1BITSfOFl--95z zpT+hoj5I976F-Iw*P@HAdVKAEx}z=|0SnnoTj$-bsY`-gnu*(R3EI{W@A{XUgY~ez zfopu

IwwC$1Tc&~cGV{Y39{qg%NCw>zJxFn^x0V&c!vFL>daG&iO&_j${{q1feY zlWo+LsE=4BMp+10X&dr^PUi}zQsqEz$G){`FeM%ODBl>4$j^twnx-1UTfx^%RVcwi zdDWHsOkzfmCJHMCUEMk*#KipUY8(0U{QRQzE)r#*a$j^i4GKyf))kjblk{k*d^3gc zH3ezpITm91!cejedfNAcrvzpuNiVmA_AfKht1Fcls;9e|Yv9M~9nWzw_Ol8d$M!7Z z2b!i%eXQOwmIpwRVBWcC0_w+xIqrz$VJuTF`)44exw0n5L)<33jL+2+>+9M_vITob z>Dgz6={Of3{4SluMW>vSBZ!NUua7YCftGfIeuWLP10krtHuIOdE8#J*l_)D~MEg!`*(`3bll>%Gh;JdVwvWlR%4ej0b&Z_m|P#7Q*D>yXKo~XCY{{<#4QG(zJ0caQhwAIew|Y( z6D1encNvcYF71NpFZ`KZO`l!c3Bh+U*)d7)!W>v>E?&)lfsM4YiLg*bt&#cuzJ!jF zKT2#bV%RVDI4;@zjZq9QKHX{PEqJ&Zo9bN(yG!BV0Bq}@4OVvZi{kAK@C#g);vF^O zRy6px7Hztc)c*dc+ z?V>BI9+bHLQ@FU_y?-q`d|mcDg)Alan+A1rVOJ0KWkupUsjFgmk?^Q2?DG(v1drp~ zZ&NF$EO7+V+51jyaL=p8`c^xj2=SK+6z&AnXzlp6QWNsqM^COg+-ft|6d@d(M5hw* z5My(BJtCyoN20kDW4^sTU>ztz?Na%^JjBlwEZ+~hLNkk0U-A}S3JbPI$FJh#W`SuFQA`7zYP}$LMb=NMW6YI=|@?nag@uIOib# z_76m)%t6%|k4|-EN~l{+0~~wt>ZCG55;hz=9v}OER*-dyEVQCm4hsPwvqY*J=U5M7 zxkXS|dSW+Ptu!Lv=$zXEv$L(@9Tvk*CZvgzBgOF$C_F331q;Y`bsI6{p=h1qw&Ic| zkcm+Yd2W3&5=eKvB+c;;j-9}`^%>K&$`0|$tkfueWAi7D4F%0maPAHjp_=2y_wT`Ip26Q;^xBJoINFuz9xDQ?UUJP+gW%JY`xdgei?!$iWtua`rc#E%K z%lv-jA|h0rfr&NfOSFH(bV70X|cC%%#Jbg)&s!^7qJL`m@VS8XcGtc;C)*4H&f_LuZ z+aG(2i1Fu4cFL4oSph&O`7=P2%cD)kqloQdTHuAW^4wJ|`Y^?Nxh~jwyE$$C>b)>~ zmBm7c)gssBw`|gqOVsu5gwa85fy?(ajV5Vw0gv;uYb9YLv#^DKkt{qwH(U^>WJHg2 zOoN;S4JhRzWX$NLlif?toL?IABg}u7rXrlPiL=0Xs3Ye8eAWEz<9Q6sYV)=No-0+g z^fbn)>QaR5e1QC~S7JE3ONLD(2Z2qs8Z@j`-GFR+-n9NxPC;8TZpg=mr*DkF75VYd zA;b)duesywCdnFFoNe^J|p_6 zj**WrVBX?hX7S7sxftI&TiHA*y+ObEbRbXlk| zvV`JrgVn*~E?KULfYwar(ZQWXmGs@1{^Ox`QO^QS43EBS-&*!#%{{{kv8Qw74|N`* zB)@%iU>=Q9zd)U3#2+cAY*3_yT=T7#yv&s9gCEooF1J{9CSjX~ZwN99%K!BOAYW)R zAoKQN)+e)JgtPY-3GU}4Xlk%k@Aj?iyN7q+yT4AhOGT5!GGXN8VOeVjQ&cn?}MaMwfwj0dg=afXD^ig3{Yb zHTUoc18q1C&zcVBs3v)+?7tdEHzR{h(2T8;70S5P=y$fbTic#!(H?+yZ5x43lV;Q2 ztiA3=z|4g~tOBr<&u`w2Wf$kI1l|Hp)>L3qj6f$KFb_93vYa?uD3B8{6^yK21M^o+f1!qk&MIhX*fgnO83)J1%u84siF1p>8$w z%ALnQ{TvM#ftj{{GucUnWoOY4eyNSg*$xJLtdVSe`+FSAGypyrb0WsL z*ZO3?+H7ucHB1m0-x@B}L2IGE5LdIa&;ph#m0rtpcmGT7$5ouTi=g)99JXUfuS+`) zcjAZ@k3|M2P_HOhlP9;onH)fUZ6rrtc3{Gc3&syh8Vn+F)&q`pjJN)=kLPw|B|=*9 zS`XVf$;RT?;(HNh_7j=VSpXUKbVGWiBP}RYbFmh-oR$_LW-Nq7R#0 zW=U&OTBP7>0BsGeBPTc>hnZdUCJTbNFKYyO-431n)pD&R&#N>sXCb2}hhc)PuclH; zI)U%^5D^w7aliR*|c@mxwVAWmXRS6G%YE2$lk81vi1icvd z^P9Og3)RR2EXr31|Md5X@0aLW#64-`EE+tzt+CkwlZ(!4#O#B%=5&VzIsAxK?6%kh z_Hnf6w*_c(&@G-n$)A9wYelPpFs#Fgc(wY|tz2*6i>_6Asd}d_rH8vi_&g$x;jZ*{^DQ!O!89sN|temo_iN-4* z&;RnF=8gqTDa>23Udp=V_4{XGI39ZxQzS5i%JnpkaX%-Y<0*DjG6LKDFUDA&zBpdD z`aTj7ZNL9p6GJsCQuJ4i%(-BBRfO>+bnLC|TyU5bO}=(=;&jH^IesAS&A24Z;C4WK zz~D{Utm3$4MI371Dh$RzZ~!E~_;3Yo5h_S}*t!9Lqku@y0v=V_y#E;s0NvJ_g)K2`ffd4r(bY{09rJ za>Dq184&_Z#9;aw3ve0Rm@=4g?%tPKa~K%}*w%E#9lh)GeG(HC5)kAk*q`h)L>V+* z1&z;~;qD=zJEC-4V-fM4Pb!{Z5D{(QdQ8%Pg$S6V7L-0IG=2v}S`L0Lu*<23&^(bF zh1Q%SMF`b}n!JjA1Evr z_tb9m+4r92HYUinQ2uhlmt#lRj=Fj?S>9$cRFk=byWX+e>9l9{?<);IVDr^f1Qhik zUH2P`}Ek-#fFR{dT_c-npjGw*VMSiTwYn{O=Ifk^GRWnie$hr>^ z#SDy(xXxiwW-@cg;ZR?h`7E6~Z(@=o3Z?+8ISq;@@9R96`HDqB`X5~SMHQ}th{Q1Q z%2pJ8!mPOcS-hhm)w+UtaFTI^n2SSsCK0UhC!?wC!E70pA4tJU}hlPMmPHZ#d= zgOk&_RK$*JT!K;>yDg97zy~Gc2~BobqM*H1$oQhukKhEygzc#bo-+Je1F|n7{H*EB zQCwceiHeb@hJdYoOPLC20QaKpM;YtxYWc&jj-i zBLiQ}LBs7CN;@Falbte@YI|$qgKckHTW(H6rlok8^A5OycTJ3v+2}-TwE9>8qkd-N zE%&vS{j25qyxqLmKJWe{Dqaq@JO;-fnL9iOW4+J?xm4HPsQB*WDkRFIzJpo@f!>{j z{9^w&tHB@lM4(i0RXv`meXZP4PW71r)+=Yhij<_qgGmKdMbVfK3-&X!Km={EQI_0c ztb9&|J4lr_<(5k`eg`WB^a?t=l(0rI$#RF8%SfU%9GRtP8Qd8*Fmkvk>(}*7CYb@Z za9Bz4?%;bi0vKm!Wzvy^`OX|1bq_ zA<;g^dH7w-tMADJW9XFz+0sQRN-RWYIM-iy2R+O>SA~a3-AR6QlFME%!QyDw#;?sN@h7K2|bklQIf#I1sInX;6 zMwG1m*ofdqHinQgRsLykim~QP%als6TvXx}jri9|wK(j50piC17l}7sesMXNmgIjh zW57v{u1f^WxSd?KGE5h_u zt$~ZlX{h?0^4)$dS0&(0%i2(I;|YPSx4Xy;R|NUEcHv z?75Iv;SJai-{jG->Z5%GNZM0T6F;Cn(4axWUN=Ne@GU*_H%vQ#6t}6EA+TDzCrXAP zu(&)b<}sOZXnfHIv{Y`3eMsxBr_3ZXv5ezr%?}Ypgv=$rK0g&v+6QSQG5Hrf^ejL( z8NgMwbqjrht`yKeVHu#sPDTrh(lR6r=zp^kF^HoNFH*9@KA0`vy+|iXux+?Qi}+1OiAw1);1c2hgQ!5%QM8F@9BQvbzsmf^ms& zik0*q(k_35rHAKRehMoD=Z#%bW)=u+msyXYli-71KAo(9%GIjO?VxNB$|lVr=a5-8 zH74Dr2fJOzhG+*UfuDU7PRhzx4zu#==7@}`&;7m^Nv`;#?}+_w))|D(3FeQBAVCgT zI*bVdwQJpA%#sQ|D`!C#q{F)kR%p87w-p?pzI?d8fILP1^(Dx}?IRBw-sa{Rm((44 zi}1NBKxwA?P}mL_SvHMnnZkrwq1}Ps=8tFg9Oq)1x_4>`2l$@1UgGVnCc16`i zb$5>u*R|r4x2e~Q&=DjS0m4JW0pm%)@EJHPxoa>`UQCNmTpByO@SoI99-pb6wmJ9-OT+1 zF+cRW4{ID8WG=fPJx8}g?yy`u;x}Pzoy5K>MtowCZ6o^!b>LbxT@%BZ49#VwC-(l; zcd*RB&m*uT^Ue}AnTL+R{+~&Wu`sJ?js$xDZ64SdwpQ8^CeZhYRuutOERS;uOS_?+ z$C_ej(8YeMOGI2YaNf!uEt*dX`-EmtKF!v$DaiL&Odwkc`2NY_dF02GtUldrcKi4{ zHuS_J-h^>F)nUpEM*!BfMY9^= z)p+B4;oz|4I_sb{%nvJ6%om?A_U?2OQvDbh&$&)gFjq8SG%wbp#`e>chIS0|474Sc z6RZW7;G5+?sDuisR1}9s5XqH+$2ropTUb{?{tHSeh>ip=;>Bgv{p9IKHwl2m=wSI~ zQqx3l|Aj4zpE1gB4XVAcv&s9em&Mn~-w9)`csecs(!tpdf#?|fY`{Tng2^Ilk3LE$ ziyKXncB`%#scq8k4cF$PNGohcQ1fZDQL6YZAA>o<* zLlU@n8aQY|2=f)ve6O}2aYwcbmpp4?BZ_dXpTkj>JlqiYuv{h?q*xJQw2@XdSkP&8 z9+?28rji36crL#`)CHB#2X3(B_>f|*vKMw{wd0PDO^(q0N%6Ew8cFQ@vK}-ZGH``q zT=a`zK$&qULb5}pfQkJ7;2ZxoHgr0w{FiV1Qtg9PJ&xcP`EHe=j;xI}K+t!EkejaB z4<>@bRB%{az%`iE+vtpn$~!~=&Zp>AQ=EY4Gmk_Bo8y4uwX%wXYc2XUtNw^uas7A+ zOWsCdG6FflV*O`8a$G$k8F3Ht3(pV&_>7@@SSz@Y>ADYr&G{Er)~v=OM^Z(GFGxaa za{TcS(eo87_-8dqHeXwwn1~SJl=v>O$;z2Ty%*6qubeo4M`+_8iOlS(ln1{Umfob( zh-2^6E7@)4v)%pnC1w3aH;UV1LpLCA)PPN1$f9T=x{#J`bz>d%MV}YdgCysb}pQ&@mYms+!jGk#GMz22xYL?MsuuJou z9oPL6;?py5Whk9qK2L774QSgg39&%_r^YSUtX6O#uQBHbYo`ttrK+e{s7_h`&% zh|5sTajQ71&0*^Z{eQWTe{?ZVid-6r#not9vVRfdJE83jA+KOt5*w{eSnN|8TIxhLZUW8^ z28&hq6r1V-ioen2Z6fF>GCk5fJ`DS=V3kVS}i)<-MmBUW7)`JcCcum?-&pQD8> zO$8{XEakqpcz&q+13}w#zwBEJtIsFv$;gc7zc(W@Ro`c8v$Rhl{43|sOC)jNX11|0 zN<-V7BRNt^NTj_Kk?j#F7#@xSI1_OTm85yB3E=TK%3E;zS^2s}rl>a8I{v`Y_GqUJ zw-c~t1VOwSE)+4xA*402x3_T`-L}a(Cc{gP>CMq)1*bBPRX!oxvz~j3wC0QXjqA*@ z=fpYPj38YYXBo;ntgF#P2NkpwLAD^eHcVZA-U=2~b$Ar3c1P^z1cNWjiLL@?h34zN zyMbw=(Y+TGHrnRbaY0%3;*xwGgY9Ot3CYr)!21rY5HX%0YKy=Qrk@=)S+y zp)Vo7?~O+E-t}xSeBn*#>{(s7N)6wO7cEprWoC|=?LmbteV%n|ZusbI^eOBqfNjOL zZ`BTH_eSu`6Lc^clYWV{axAW)WI9=JjC8oWkm!3?C}e~WX6>4rY=)uKK;dOo6GY;h zgz?ecBt_(Ec2H`ZiggG*NBEtQJp6&EQ#KZ!eunI(U0V?GuHL)}P-JyKimngDpW^?f zo*n-Uzvior-q*gO#YB^L%!-3#4v~tBIro437b5>Yw6d>rG#STA!8cxeL_hz**<|HK zBs{yd3$lw^e9?jh)?Q(n!dfGx{eSGeXH=8h+BUjuU;|V{M4E^g5di@q^r$F^iijvh zNsLJUr)je#-2esAE`5fnS$h+%P=2V7R#|GEKOop(-56&#Dc9W}lZ(v+D5K^BA7Eri zIfuSs=ap5%oesxht@3$QMKA31T!`AEKiF)x2}=3_%#zQciWfs)E*<2R7|x$)4g*RqbDI?gX*_UDGfJZ0acr17^~reW ziV~TBS9h+K78n4T3Q0TC)Fpz>_zi#}E}g;e)OVeE7X2;nsULpiw}&_9n_VLZEKh#m z^_M#jiJxI%XAS8}8=GjCN(AFm)FMxI_gL(z25dp?GiOv2WoPytRuXH2pyY;g8g3h} ziZ~)$PurQ7a>OV!{B)N<&ZM|41H2VADI$}{THYWAWt>@Sf2!x&Wwda*9|Oq>=4}LL zVbnI%C5DEvhSfG2Gh9~hrdr=*3eFaV!OHqh$exJa9cu>a=%j7TT@l+T_VQlxvIB3P zQ|QH<0>zte>AiTk;mVaA;S&vy%#_zqb28R=wVZR=ds726*E*@k9(&M@pIwg>=No0` zAzO=j67;1bRDcv@!6V|F5U?qj%cd;8u!v=4;Ty`s3P&7NZUb0DV(d=3%q}%79Kh#n z->q)NJyQ{h^uJotP{^Tb^rW>m4Dq&&n722;d%+F)CG?>j+T!NWwhzoDbs{|ZE51$! zTP_m4^kc}=cCZ=~_@;ARSA01Mctc#tn*6Nz*_4|DUyAdOrEfMhlAP{8w}J%yo6Ek^ zPx5yyTjPv{xb$Pqu`S7w#v8#QT$^+1+j{1-%H5iKPVx%^ku>kHsTc2st(XOrWF1%~ z3%(fKqM;JR^5vhIFBUd)(c&+>9<5)UlM#6^caSa2VdqBM>Q0BlBTPTn;?9z_)CELV zW@q}D=_CQ7q{x)kRIuWx%goYMZV#kjal?+9)VBM@P2bLw^91qtM%R4+tws{Ms%b%h zso8d*dfN1YZ?NeniByfq#0#l5wQ1AaX@y1{7CTaP;uss2ZJMR{Z_m?aTVZ6+`HCiY z>5KWON)@?>7dJa|=lhQh1jSk9a#5`C9Ud4#OfNgdE>dv->VqCB5{Fjb)3iXD9pW4yesEb579; z?m`awqTiNlovtut?Su)gobO5YSpx#R7V!en6cJYCde| z**-@0h?tsL`4WAg_C&*j1f4CsExYNs_F6MbylZn;#@>wiJP8P5!4q4ScckXWp(W31 zog9;{ggl+@3e0NqAH%OEul20QV3MCthO|6#W~+rwJ5;MH<*djF7g^0ju;XP9AUc(( z!M*3wxAM0a#}owHMl9h!FZs7DeT_er+5I>vU-Mk4fW5&>h0|+6NR*C9amHynrC9Ic~i5yK+G~FTmuy+-E6v0f_{#;OA3*e2C|)NdjY910_;cexFk# z#4Xc-Q6C^O#^AzU^v~^#0*n|8akY@MIaNFSg|wWOs&ist!1a9K($uM!sV*<>AcvN2 zG^cq}+WO87@zBL(t0$knmS#6H@{j9VT>pAC{rq+}QNjI2(!FcpzL^b+LfK@8+$`9U z``RWu+k;%+%a0$8fl27Jd_Y@hhmXGguJB{QaIJCU{q1|EucQ&3Lt70@NjU6z)L%6GmLI&od4ze<^0uA-HJ`7p zw^oQsn~k=SEq5gOY{O_BIvj_-?8t8gFlOGrKQNa#MnB&?uGFRJ=wA9;WpaS}IwUM1_M*)kYz&gC!MjP>rPWN0(ti{Nu@v(D$;V^xbe-=%d z^nSfqHu%#=jyG;0lyKrv45)u96yJVMwJ3X;@^CNBV$`?cl!-5OyTL)pfSnPlV_Za+ zmwtFT>UF_LDynQeW^@)ZW5>k2ZGd zn3-@RoEi$l*@kT8l;`s+ovf`I5TS=2td=*6YS z77dFfUSA7#auJK{vFHRJq_A%iX}SrRcp;vD2zF0Jt=sDYgF5eTfgZ zoBO7hZ%F>mnLanX>b~k6O7G>PhEJE$Ed^>*wIT)21h52@A-vd|tv2Syq?&pKzGtw4 z9)T{y%VPVTvl|6s;yBKD>8X+r)_ZY|?Jz#|u#n<8g*DjMdyIy@t{PQSX>nqll$lyhgx zM!~=Tyc6wEL(gs3k=uiv5wzD)GJLnfank>a0E7B<3C*QV;|EhKhqlOw8E!emZ5P@=QZce8jl=&jFw&GjzPAN_BiXq z?6{JXeK+Wj4km6myYLn{fua-t69g2D*llerwsU!Nh>~iBZ zv4;gwTa$%1gX^0B!>`1yl3+vwrIi_O=)0BIEdUZ&oerF>8W8{JY6@sVJLibLswU zzxSsHn&Tj?jYyr5Bip-lTFIGj{st=!eWLx^YI8SK5ps8UKDns=jdo>nxXsJXgXz$p zb0<_~S4-05wLn$Tfn9t7Q!1QGc10q|x}Wp~;sKz`svSx-$bNg8=0=s9iqwVl95?eM3I>kgzqE?=@avFKkdr(RPQz}A8mP~ zqst*yccTR79d9PxBOhK}0m!pp{Jy*tuZ6$a2>5EQmzg#$3<&NBNZ5Y!{s?*H zT|(91c9UQ6!s1TM`Z*DXfsC-3kJ=$D$;@qhx#k{l0<+lcnSc%9LNSBsJA75r3g}0F z1NkEj#y8H7WV1-9C?`5ru%Dr|NSzc;ZceBvuab7wSc`s;w>fupK;~BKgo<{%O}&W_ zfvb?irK&GlNPVrdtJeLX@c|o4zC|f*J&PcOo@#bWUbQShdanDhQ}&&o)0MBQ-L)&| zY7*zuAFh(-!&OmIAHg2#Qw3PkM4XdLw{X_;{DlS$qd8)qrwQw9{rlN&yXB;%_0;;U z@%`4EU5+F6R;bg_73Qk04JpBqi7l?G>#1KVPdBZe`ypRcpHlTm6{ehJ8&te-VzV-Z z<432J$^o8D`Tn$n8(fx26EK#^Wo6T-9|DHi(4|OP%GTX(ERX<^%de5Tbh`3fbF(so z@e#`#HZR%QDaw}(zQ3^MQgtK!$uTxm%oI$q5WF352q{o-eJ4-S^eqkAynv{T+)P3P zE}|5y&zSgeZJGzTJB?hY`M<2#UE7YAH9Z~PR|2?ak^>e~w%ljPOx>;BLDqGCIDrkJ zX-W7ZZPHGomhK(8caM+Pa7DRGdZ>z^C{X(%IzY_}fPbfVym-h(Xso!@RQ{M`|)`O6hHUwc*T4`AcdE2kJ7`vG#ddXn0wRbDfhK#jA* z&Q|#Fp!CJ=?!1QOGvC@X{ zldIv!35yC(9IF!ffm%g@L2O@YZ>uGp&g8##?(O#vrX*g6jBYP`b+w}_nTAPF#q68e zlikLrm0N@vBEX&E=BWrR5H&9)r?ylj{rW?E5rxftKF#&1HM{D5bLcUdxYWeovM>ho z5ogMt&fTigd3QGm8Ys5H)0;Kl>swkLVs*Bx$}~YhMLrXB_NvSABk$Jlg9gtnm0|Q< z*zqX6iSG2G)@_yfh~zIk=ics_6mf;VZCCmv!z>toD0Y~dnSTDhYouS7#`PNoQ1|#H zwk3XT-Od(JSF=SCTXB+inbntNj})D+ulz3G*W8v`Z6{zw2T9L%H+D|BnIq4%#usqx zjdrdV2YU;CvtLv(=m=CmT@@!j|AW z{*ZX$v^(b~p;T>3)R{7wdZe%fLGINbUE1I<=soY72|6M<}Jh-}WIojV-Ny0mTK_(v1+y;3Iy zOlWizT4G&*dpe^m!2V;>AJ*0HR#(B%h319oPwXPXzvK2U4HwOL4^_~9YOml{4p0D< zJpI61)J7x6Q{w(V333noW9`-~lArW!nmELI#%6Mc!A?29)5;uEz9S5juGF!>@f0V; z4rPjOJH~e5N&uIapS%$~ssxQn7kBm6Y;)qY9|Y{!MN!X(39bMX?ip3yj$l|zdxmN; zuiO^)EF8c45F$Eba#HRggG$U6lN;} zas^SnX`Yu~`I@}-jyO=2bR^2OA=nt-+pJ9`19|ND5qluJ=)kJhr&JB@I!N@|y zY_0t``%-z@QIkogkP9b>RtJr@H*jW_QorfpqqQYsxv-j}UrRqEjsQRlvW zJE7F^@qFuNw3=co*2t%Dc>))C%CzD*20$tFPFp|v28sBiA^F>kxu_sfB zV(ReCTvrm}@kEC2?avLzi7QZJFI9n;0~2*I8m$+#tbAG%?5cje-VXjLKPJ#59O1tv z(e3R%Y6!CAleK9RraoE|vh23_kuoE&!jkUNb?veBysw1t6m0LyIsv#w=hiy|iP`b@ zTg!!FB3Fb7?-nxMV(sr(qP3XU*F)O;r7^AEss}!*_*)nV@wnJ8?1lB8f!Wuci1e@( z5!-34%}Ea-=$juvw)dNSh4^i2C;5QOKgL94^PQ|+;&ME9^ONE0B!+1lMNoH5miKWR z=lYFw17PcIOsq9jcS>;(?Xw3RrWvFg^Hw zy;ttsInx9wnQXQ44=01&*nUIG+c#MbieTCz)2Ncm5h=@7ArT5O_%n z5R{=oUkCwFk3+p`bW%sS%KcviDJj~PMnpRX8C$1E4%+~BrjHd*3g5iSa91z6;R~f<|cLTI@a6h?#iz6_2my|wjN!ck1NxNqGTS1?PR24`_oW8Goflq$p+n= zHAH|%rg6JD$LXdR7^;WIy@~dkGaq|f+N=PYu!|Qj6V{a1)>M-7sqwy34}TOTATKZ0 z%dR@rQf83tH6L!h5yb2A#_xfPI&@8vr*`CsOkSZK@~QWFV~_H^_r%K4x`yyqFFt0GSbX+-o5-D>SOWAuSVDzUY!jYv-C06jYaiak*}O!*Huwji=5mmrF>BE;f-ygueA3!wqlLE>mi4ZS7#__>y__+ zyYK9B4)nx^H4EXVVw52TGRC~<65S3S`(oZVzq2AG^I6+@V=Vs>(snb^(ABQ+(&1T3 zXo=(C+tqVgoc7h+DI+PcRv@pz-6ptSz_T^-RkuXFY}#<}Z^~ElSd#ood5Xf9I3m=( zB)|%q?IryljJ&ICmAX83_r&ZsV^}8Jwi;KqSGd-SggS;#twHILLR)3$3rZiqbbvg? zc;ZA$D=XG~z(i5-ji&X@#FPo?vFbQEe98CBTcN9edUlmf)IsZHh04o&Sp+_fb|K_7y+;$b@?14}-Y(zD0S;D$8;1lx z%T&(@CMdFqIBXJJJ0+)rZ?|`Yo_-F93+b7CapW)X!@%b7R^x(V@~NzUiLxRF=LDeH z3G)VbrP#X<^nzdTYlT+O-Z2+m8KC9b@?Ac%raNU?8sxjedpetyLH$|-ujb>Gpw`BA z?3P;ZK2u6_`Hc2}oJrOgc8ct}yme8o;%=hal{3y+h+)xHVe&*6vozRvcZQ@1n-BQ#bGP zJm=+9$-+5yAAZEV;HuZ;Qp26L?bkN>{d zyo1ZGN##e5fTAlLE1#WrTVId`ttm`%uBy*nF9|vUOg>yE z(mv_U>H%C-6JcICeB@+?(Z|wo6|-^kcYI0_Z+|7p!UyU)Otlgg;ZtT;42^1Q2)XQM==Ux=$ zvxA$9VAl(2ra#PSM)2jR^Uh|@x%a|unaGZrl#;C&@B3G*ZWl^A!`};0g?z6p^Ci6l z8az;qiBq_>HaEo+_@fo4GWF==mc6Taq^rjHa|+q=&w(DxMJiiC>XXu8q*x$P{>-?LsSWV0)|Mt4SLbBUv{^ zd`^!xo@!<-q@6CBdUL@vM^!)&EOC};oFu16Q+i~9ai0dLoQm00DNPXhW7{G z?B&e`-`Up%w3O3HwWeKP(Wb%{!J)FD)JZw8%cegxjR&+59_WJnEP8gHrkAF0QN3~b z<#@8;Bbk~XX|y?BaFGXG15QGEZ~Tjp zIiIUC<^q@E!YIw4Wc~HGvJr6lw&y<-dzJ?pJ_Y!_Y0(wCEO!`_}5ed|< zzAA>6yzB!F){j*l90sAP4(%`J@LxlT)pvb9+U;}(cZA4$cX%M#32X4tPY70eo$uBy z(Ai2?R`_zH3{W%Q{8IFZh)%rQTyy%8MZV#Mu{Z@qW{C_I{T@N5T$^cc_ATq#iR0bP zPWSM+`s8WgL*9 zvwoMpPi|#&;b3X#g~|5~cQo>`;dNag#~HSzayj;1)dKx)&-wSh-vNm{16v9l(Y!kf z&hM6-d<*z~)Ro|V6DDc}Z&HFi7y!42|grXWYR|0PHyv{v=eM%P_6Yc8+p z?Y997s$BXXn!0u}?trdbR6*YFb7)c;{E?uN4ML(kEXbR2?b)CBf}#o$28eobtc;aE z!j_5iZcY-!?Y(?n4vf*aIF=Nq3Q`dBH;q**%ZdP6Oy8{Rch^*_iN&&mPwUKMBn(1M zGyQ6nM(4HPO=;?=N-d@;OU^hfZpA7-oYLNzK0@_QaGY{b=+cXsfVZ_sIwV8TA-Q$^ zH*Yb|=KNFtJ3k8JTYs|D85Y@p`8K@3%U5>}C|ezMKjTDvU1Qwn>~Ds|kMxqVeTV9Z&XD9SzbDx9+tR0_Qhcjtx|%PMDWgzpaSUY(xxZd>=PQOhUXJ?a&`uMm~(|Iz6wsY!=%Chx929MY= zATR8mkO;4uu``S8(pqtbQ^%%e1RoEdm+It=H`|5@&(C2~9pciso{&#gZuEbUhGbNu z`-0r%o(Gl@ukxvDcYku2r)xBZ>Yw+`e%bCq7@>7e&DnA|wR5J)pl1$gSV_g*P_6}E z0vtA(Jin98^e?W8_@Gk+rFK=r2c+y!)V*JI@0ZwonQ7#hN>&4W{2nY$rC@}1(8bUp z-{A%-H9G)Vzj=2V-|SkA3wsp?JLJW0~e>t>ooT^*paG;r3+(J?4oWPgk%@dhRvs+EU>O7(z> z?$hUg8iRF*U35LBn@w;Z^-V_#O47yoeF(erVH8LDvcHMaTC;uyKhW60=*+N4-GL;K zVN3jx3M-Q@9~Ey-+WDfzniaOq>}xp3KG)gp23}i~ud%cWkk&8fkXcP0fC=+N6UBj_ zG}_o?Y)|s4*Jk{7!ZEJ$t}4|kI9Yu7mifVj5T||DNTe$Hb{DXnX3&S=+TkI6FAr0JI3mt(Y`Kxh)< zW+~?j>I311`-N| z=X$VW2a^TP7!)OI2rQO??Y2s zCYN#hXGwIP9r@cg%H+O@&o)IqyUK+6bu;wvmAAwXbruNDM6hj@n+Q&{!QRfyw)<~x zSO#2FVEHCw4chVFs(9e=gkM9+R^n_ubJVY=N?7Kc_&c$DLaIZ*9hV_)MqwGJPFZvu zv>7dyw|_%?s73iHV<+jiU>vN<_T>y){oA|T6bE{96*1g2*ORK z)TO2C-m)GuIz1>7vXsetoZLVFJpIxJqhDM1r>{o2Bu0+oLu~LRJl7fms~Qdv?W*8W z4ptZDxP+~u`W^~IO>V*lcgag|s@2)3flV*fk@h!w@PiKR`8c2y0n^l19A7J-O*ezQaE`g|RIDkxCQDd+NYgSgt7#gFlGP z3({)ke|D_E7yXMw^YgBw<7G8kJD)Ri5e(+!a`Ok-r zlD{&@p-G~%qJIwdZwvJoi~09M=2Vy+@_O5z{G}ZHRb=Hqhhzr*mHdF~cqFR*mjV5k zQ|8}^O8;&I{f2B$N`|ba& zF>Lw^xh6~Nw(I`GIp-Z?cF@a=&G<9N_|N73%jf&|C%?8CsE$ZTdYbkR=WL|)tHXuV zlmC<~xDQrKRUOUk48KoJY}j-ohPxsPw)<1Fa$c2k#0jH9 zb}aN8OfTc`Ut`3gRxM}qWiU?A z6g6w#*4eK=iPWFIH}KWjNKasIO`{^6k)(k@ptie6S(4Dn_XQa0{UxE5KWhPy=F0_l zGAwYbd-zfultAYYTg?4ijjsQN^oI@wuC}N5AxTbAkPyKDOQo3)YUi5$!^e%)2tPQK z{4-XAhh27v=>ENp?fEiY0md*_h<9la`xQ=?w)D(M#p-c-W9(=ccW-x6l`|)o~ETU zmUBjVFaa!d=J2up$|!zljnb$VGIex*j{(NHGw&+%Y7v}i?q8*z?p7M?T;5Az1&B3F<{sOCGxOR8Sg3zN}mC!4(;tM11mUSGlvG@t>Tao z0rFb9op)`6E9t#hK@ms>D&US@Mx#geX0%QF$%Osi7V+!kmU56UEgjL4#=QC7L{FYF zhR&Ts`;gMhOEG>%xFU148Z0v43zJ8S=FkcIw@E@A0e9)kX#Z>yVl{v~u<~P6aBp(8 zLf0{+!%cxU6H-yLzYw*4g{e0g=ctm8)r!8q9I##nh5%ArC}bR(QAT!;Qur|$OuWa< z(1+r1_;x-D0lrTj2K(Gm+F8Jrr+TAw=u_NiGq^n^oIEQD4`p!p^{9=uarhX}bKpBy z(;ukJB`TG5Nbc)FH&40w~DozJ_dRuIzQIv~SW`U3AyUc2u1&rj;Jglw>Fn6sCyGu(V z&9Lsh8GluCCHbK$^)`odPAVDn9aS?(dmThcf*r6~Jx9{7uG|CxHK#xX@G_con z-H}-HUCh;XOa?kbY|^$9?5_PYS&NIS+un=pRTA{ZlD+pSNH#a+y|uTJr8y_k<>kcV z(y5JL2l(ck_u~K;PFH0Ca!>kfSA!&s_>hF6)M(Jb7}%8>%BVvjIS8SW4cpM&9}8IF z4yX>m@9%jh(89pGtk*%)JeWzXcCo_R?w^31 zZ@m_$1qMwiJNl&oGtr>BGN40lnqIw|WIl@u3TF~xNcogtn z#{MU?kCoBG$C;}q_SZAX1B-&?lv;}5P+SM^-eUPc4SCdo$l|`SF zyS12CXfNr*JT>#GtA^yppM;0#BR;B5C>LK#tm| zS>Z_Pfp$0olwLz$ftG|PCjybgu2CHg#;K+0WAMFmKuOoyp1saxrsyl&4NQkEE2&UM zprB@zb-cF&u&OPPRN$?Wp|TSl4LgU1Z}#RWcltX! zf*GUw-`ru^mB&;-s`L?U-c{`)-}00`a^FKqJM`WRxP4JBfw{BhTeNx(BpNv@K40X& zq{cDNS<1@i(sLOPM7ioH;r0$oMKV`0l{6D8I0OmU4Eq1DoGwV`@JrNv>TLgov{OxN zI((I)?<)o)xoanUN#f41{<|b+f}cQoNvHa5?q~vX>$!im3@*M+D_O2QXQWGmFMWS^ zel*N<0PTGi9>C3-M&6A8fKq3bC$$mXQMlbY9mk*P?#Q*DC%)4~wP_!(Wv(Toef?_Y zRrCT%)-hm44s)>-GRCLQg_PCs?`Q5-ZM2Kw9KaMCQhDJv;0b^o`=l+o5DP(7nC$LL zKz#bPxC(s?4GZFRuf236vu4Rn`URq z{Uarp>UGRXmRWt(MXW+2t6*%5=Kxw6N znt4PCd!%F%%PJTV8?Z6s)u%}jD;F!Q{?4dnBoQ*BP$tl|Am%Q@ms3XzYF37R7m#QM z4qdj1p#WZVn6bTOO{v|87%rbF3qFOrdjt;Q);>(wvSzelGNZ^j?&~@3=!%cposig> znY;f$U`)`{erIaY=OK{=vzooV*5w;o5Vzl+<2~^N1Y(r!zVc*M|6`>RHlT#7GEr08 zftBAwCP06di1B|wpUX-pQPJiG@3Ng+xA6h*YM=>w6EsZu>dbCw#k?uB0RJ2Ds`{D)#ijSaa8;{wlw1MmGXDk0ovF&5MTdn|rV<#(pkb3s#kD12ZDKeh z!bX7?fv);lWiv_Ji2|<+&4zQ;XvPRb`&J$(5T6>^$d!>g5AL63pf(gdIClzil(9m;rCQ}TWC+K}fuSZi2g&_dq|ddp zEdW(ovDdJ> zikti*0CDXsm??TB7V`cKXZT-~Zst#@CqTosffHU$`4YC5r?OI}A%pi_-J^T0RKXiS z<26h)fhh?$8jCl(m2g^hCRCLiXLw7Mm#&L@O_LD9kyX$?EI3Ex*X$6#)XV`!9uQSA z2~~?$O7k=Eat!VNhK^5h^Y8GN=nz4NdTd(H2l?2fGgwqbYh0%zsK_6k=FJ>cwY#&D z*#g5(W0ldH-LN z{57D#E2&T1**7m3Bne$_;+9;3thx5yve;K~NU7p(Xe9KLiQ# zHGUbu0yg&){vlZ7sQasfmTP;`KQ#LNF8`_u;i?xG`wu(FiTpxR&gP{2!x8=8Dg3`{ zd3FD1?m|)ji&_#2o92Wy@wVfYTToqm;iIYXKjk+He9YK;e9ntDt6#Q{n(31j>Oljs zZ!{nL&tta+)1KJbgSzt+vC$nBC#tRZM}pyBS86xAnWnOYb21Q|-1WBvynmn7qFZJm z{9cYsGt8l8XE9G7sqh`{EHR-Q^dBdp?)c#X3>v8K(yIqMd{+DYV)nuV>VU+5fKxZl z)Un#HJKx_maw}Kc#~Vj{*!fS0ET(RoMzJ34oc#_|U!Qm1{<^@Iy!b!obn&3r={mF! zcH{Z=9m?E;!-N0%vZC<17yHPn&T~$A`12-hPe}jyAOHPH5%O0&1Ia2oF6TdH`rmn; z9w)QI4|RarKhiHx1(M$W!h!$)1RMuCsZYF_Ud69?V$uzInFyK+Gc?>h%3f)?X!^aP zQGYQ#f?_s?YYbv0 zL-8apfFk=0!~B)c4tU0w)%f>pIvWlIQzG~17GHXcZ$ z*I?|^sRJ$Nc82z!qn$5bJpZ-sLI&jtfAOJLx zvFkleib8T~rfklYj=tp%_{x|CGn(Xt+(!@+4y(}p8r+e6>ZI2!m;NwXkm6*cyhN$k zA@*ABvlx>I^%nbbJaH+1h=V#skzW-qh>cgpe@%`WaCwcnMqe0f$Gn|aLHjBsNX!0x zoF|qS!2zf4l3UfV@S z-Y|}*6(A6t@Xme96>|i0;TgG1L~Xa*xm*q6npe@$Y%)r6_Rb-1=A@`I#=XXWASWo` zp@5|=0OFT6O0t7rAYK6~6q&dYMRrROIm#Qwq{rTs?zp1qND>o@5&qaOr9nWc%*E;A zLR5DHy_oj73<2TDy|;OUBwOffMQ?86E*Au?f33ja>1%?BDBor6L3uoN2eCG{pq=8A z?lq(UuWCBqBH-RaUeI8?E~c76yO#DiV7o)kX}#oe?|rg^cEBCzJ6EMWsvrbTAtQca zkfF6B^hh+^h@K|{#tM)>q}x$YSnWP*%O|1V{>{(-Y!Cp4A2Z&9?Gbb)az59UjH~H} zYC!vqy305^O`CcJZQ-&D2?F~dTp;UsK5R!O-H}?i%#Ehf$Z>#lJUfaeG~ipu5FGF# zz@o@alCX^N;Tdrbg)$O?pf$=^rCi)gUXwJXJXlQiT9icK_-Kbn&kOhZz3Sfz&gp;O ztz_Vw(_wPunB9(?(Gz=XzFgcaU|M?uZdG~JV{}T)q^f!~zl!20 z%em(5fI-0J@sqJ&7(rc__MBfsOk7|o&Fl~{ma&+ORw*9 z-q!JZ_w2y!VzG1A;r9;Q`~K?P7W=VdLi~N3=}ARc3ZxXMK_;UbLU1EM8E1k&0f(f| zA~xr03&ANLB@K!$KKueH-ENtlaHTgcTaGd*mUjPA!XDbg5LPZ*a@c0a@$Kc)pRTa< zTV=8VyH^AiJr;Tltf=1&4He5Fp9YqDVQ+`8YrbL!TMZg`Qd@oY-r7u;JS>bT$G!mI zBpFgK2cwH!&pZeK*u5cw>?w4>n zC0R>#B-d794TLo-vkRj?3|v-kajdn=@5$8IV4H$SS#M>_I$f>g_gd%;HCnwNM8DaV zEaAykau9U%)Wut`emijRFtgEf`*1VZRX6;F)Ar?p2M6;k6*k^s0LNO`re242cTZ(6 z6(D7+J&2#21V8y}det~pL~%aeG0rb7gaBCSdB?>8@2Poo zrHm0+MDyFZGnqB3)i!$Ayh2opzoDlwpX$oOvTQ%r)OXgr7_R|*Ck%p`=?$&5Of`kQ zG(WTIrHP7DGOK&VA9Jo0* zM=<$tB{(cydtlb@?>C?OFC?xo}52%q`!Ed3Pgw{1~T&NUYC zXH$XBrh4-{HmQ7ePrWQ-?rI^p;wiDvV1c;Hw#+L+Ck=aBXTA@3@JPLUMy66@qpYId z3UXW~>>!Ep$JQ;~R{NiVGhG~uTe!h~s_r$65QJq-IAhq$bsd3m82W5r(thl3yYfW* zAn|!3j84H=pC4VV8@CucO}9SQqXisMu94n|i$%qVNQ3>?X~U~^D$@z*9XqeZ7XLje zb&Q)M;^xqU1S97f(Qf2Krz3Jr7|CIKK|34#$i$QSZrClX*w925pOn&d$%6?cSk(kD zY6~sFi&_*D(Im#ag^`odw@W(!gwF^R1Dmd6v^0H8FiC(N^TOBC*6E;Kxls!~WA~SO zzmw3dBgZRWmK{{O7yFBldXJ>-5p;R#qu_o}K94%{k%1y2Qr@|e_{s9qIQ?Y^$vcus zOyV`G52`KvJZ;Q1W8<;dBtOIx)0OFS$<817cREY8ZS`CJGhK!sWmq%nu_xqtgzv{b_AFT+#FsSx6+F_gixmg`0j#R&pl2D>d9&Q+cz;`xR zUyccBsH0J^hYFplzsUSgdjDo0xOJ`5A_^q!^%x@KwueP?<()JVQ6g*|?}dz4TbCZm zI)dKa){YT{%r|!-OcO+O2$}^)#jSj|dRZ-MoV?#{nSKpoDTw3WK+FtD@aCgk$yWHS z2fZG8=^n&5Mz{f}N~`lVNA|jwbF5qEyf)SBVdIy3VN>zAf`>7PFq{=%N`UL|oqy5u z^II>=oaIe{CKvo_R>~S6#X8dbt1!wq=MPwfA6Jfc zeD}%0Zw3ecb>#oG7K-2ND9wZmjMzN&`OB^|M$X72|6VVHSH6)%V(oa1Xf3w@9q*Bf$ zSq}DMS176Xs@6Nydb7j&@N`robhWG7i{Lr0GHSwSEUz8)`=)$8bL3H$Q7H0@7W3=E z#KG|xx^?zL%8E(}@m=5N^`?ZhFg13?OJeA04Mg*N6hAiwzfW2A6$SO(DyzyG9>SW! zWtEmQ+f>YscT_L=-!x)i-~y6oxnTaw1P&`)U#@pQG5L7MdP~|FCI6T6%X`w|$mVdK z{?Z1$<3T)%U}Kgi@+bltG!=;%#RMPOCTdYu|2QWx#-c-^@2 z^W4c3Qdln>B>iJn=^3rcop|}_hv2*sfC+J|iYUK4hy{n1Mb?Y4h2oL)Mk7u_=b2<; zry4QFRLXB6gKmj%9${XzJhkkOP8ok8f=k@>p^ST%&|~qmSlrYeeDB8<(N7TPC|8$J z$cf)K>)W#f`r&%^7+Y@USH+?_NX-hDTJXsTHsG+luFveCe#tL}BGe}uOdhpJH(1SF zYDw%(wwkq@Fm&`?p=Ifg7nG8#U7xy@6Q56`b)B)^cwfMYr_d{eMHPwzAzZ957p=mW za~@5*c>`tslcafZO!S0TzbJH-Mo*7*{%zH`5EZPww*`m4I}}52Jjq|;Hs*@MOf!So zB4vjR#Hx;0K)>p)>`5h&E3^x_l3@`Ft4KjWI(7EEqQ`6{^=HS+`x}V@;Kja}+qv7r zQGZCC-`AO>Gtril);fja(pxyqo#t&P?YZ(@R%-BrP{R}u&A8`3xd~`S(fLjZYNC!R z*$*uC0%$K6#R^ezsVm9OsI*SAvd7Dewi*1_d)W)V6ZLcpgbBHwyx#%dOH|1QuQBOA zcf`pIf}YJ;PyfolR8VT_R+AXOMDNp#&sagjgB~hMDV_es8Ukl8_iSNP;&m-*um8U6 z_m5b2ed0Q*-3ujABrs+u?*`csm}#H$4n*2|bB4D{rex!z-h~tu35Jdm zzs}BXeaK$YkuP?aT$$W1x0JvN1?XN@A+Ls%(ov465j`o(wVqF~K4L6BRaF)Rr2#GQ z{||fb9o1yE{SRMt)KSN-pd#Rm3Mx&cN_QMVItoe$B|zwr-U1oMQKTzKuL=a|(raK; zAO;8#LJK89LJJT==nz8Q6P&phobUBr>s{|!zq01P2XgW}=h{>eO9WO-e*gE8z7r-4Z%mRNs&|Ae>H)j9nSIic*DQcE3A^Qmq2Pcnc|0&Swr z)CnZ!c(hJvmi#hW>pfqBAFw2AlBMbyR_mjM>QV7ih4ptly2e>4t1uHE!YzVAef1Pn zRYO`))+vkIGxxJ<;derUp2XG14iZ9)E||a3rSQTecePr1@4J#*9WlnC*vQw*OfrnV zAg0|ljUQ+ZEWvf`morDC z$`+>jMLKeo`BCyk*;qru(7T%!{*mW2kG~!j1}=zo21!!u_-joeEJ`^`D;csfdlVxN zjnSg@cha%ZWtN`?ipsyJgVo829+;9wigY&NLqj5Vu6NV_a6EU_uEFkwmY=bMq2tII zdXq8ig1NHe*^W&s$1TFW@p=64)Gi>_aEcEBLREx*XYD}p`B@11Y%hHIi%@gUjVa1yJ7u_)& z{B6waJZi0UAj4+ujL_*$$3XXO_9|UTJqV+3W-R09q&rR=C`T?88cQNJ28yFz+sad~ zA1au7ki0nwLD&vmiZG35c8#qSI_IxCu9e=V4PJM(#_PLxPuo1%xE72p4yv0z6j`32k^#;&P!D!4Y<_5c4p09>T0?Y^L#TM9|$_-_;1{C3&SW9ela-CI12|IGNW+rPQHBdGlf)gbHE zrr!UTu}b;BHvA*1e?-OBk$<%6f5`?^x_42Tu|PHUweEH~`XJ!|MDwDY{UsN}&!9Ce zN-eINY_oVgC7&-(e+SOzY?a)@Z6ngvTuxl&b8T2pyMkOzm>Y>NRN5#zo&NVrbU}X3 zv2=kY%3zUi$-%Z1+qyS$-eUsK5spnZb;z$S)5_J3&+_2vRpQ3%o1>t0J6C(GL;rrs zT~nT;7V0rw;I{Nl%Xi}V^HV~6=}G^^%5NILfSjYih$a-c{q)bR{*l!`%CNat{@FOd zoB99OGE5{a$GcJcBnqmh*StPAj+2W8MA~wAdl3(k=F37Xq91;MG_}Ty`oH%i2l2C# z9l22LNayvUtXrfyfu=+l>L{m)n}C&UM0z%X=BH?IAag49~kj( zoQQ`eah+Ew;IPlH^1x=BuI1-iEOvE=s_^<=U0bAbaV9aSw|Di;06NL)xY;p*f?j?e z@99?tkj1j$j*!Ca>uvkD`-7LheF)4SabiQMpiN&9UL3`QN^gwR*9c!TkxMka4nNp~n^Cd&#e!L;nsurx2Gcx^>a| z)Xv5FhWal(!$C2h(j(CetoL*CvIyQl#L^$r6h4S#1T@uGX3xR1NrU=s&9X2RA6EMU z>&C7n6#N)X!B`Vb8Lv-6PXoP(YCDkm-#I*z#_c(m4{t7s z{#EmATj{=+_fOCrcAFg!oO9Tj8GTE`cwg_(qm+Y6MyoHOQ~!cuQ;wHF+Z;?Uo^&bo zD*0LF^}&SCekmeXv{P31kC-QlCKO)&>aw(;e2E7(7I})fwi{}-uWje(iU2J5NeD+n zP!?R6gHLy~Et%mzUEab&|6|v3W4nr>+;pu#fbTh5d_oB^9<$B=X}uT^PyT|HZQk*%{?Vw%E4e4{^?KofIIB& ze=uTbvh_FWmoE-44pcd`)Uu}T4KS1?mDX=;-uTgkuv(`q~cSkbD~Sd zRU!QX3fW7Nde`YP&Sf;17hPJZdEMA6CjI{Usgy>?96xocTzS4UzA=Q;8q+HNDt@Eh z3o~p`?ByJDLNgaN*4OeH!3uE3-jiwq5LeS6VqH@s{@P_eEgvysUKE{AbYn8H#IGRv zk(Geau+CP+I`W2Z2bk9)()Gu`VltrAf6SfyI2zpkG;8(=`B1L@LWM8u7=`sAFydLz zaen19y&N(0B>age!?#lei-#YPKaO2Bi>~GIiw$1XR)fLmwZ~9zE-dKWAQf-FY@td*hIZiOj z;+p_Fop^-y+n(fQUMJK(_A-Nc3NBs3Z(VRb9>befxTBzjJ^Z2XJIZ zi0zW!oV*Kgcd<|IT--XbMhlUofAzskgoP$g!yHL|ESg%8UAmp zT1v4g@{FV&6HS!<^zr#ItfNzGFrj=nSfFXzutWeU-}UMF(t~QOGsL1lN;2#r*P(Du z)`m}GaTfKgYo`qLb+X(%mon7~j)Xddj`h&w=F3Jvqfq?1rr%Q#P>Px3M(-I%@W?~j zrpL%)^s%|g$@T)OIwAn)i??Hi*$9_ zFlBJvT=#!InAwon*mqgfNJH71PlpmPDQwQmmFpmP!QP&BLq47KR)OC4A?ixOa;X?e z_;Y+A)QAfY*Yuvt8zHF@=U2nEQ{;35)DXCFl6q#Bfm+%XQKJN_vYCs-)+{KK8Xp09 z#gVc*Xq_j13>x3v5CzR-QEfxdq({pshvDGp@wvSGmtG|X44H$cq}^1D!faQcKNLWN zj88(pRh1)GdHeI;&n<}}6+O%Ex9*rr-w!zV()Q=_91rkk?r9ha!Xj+%)Y`t;YEROv zz!~C~{wJ(CWmMC*(aooD{rcpgrw?;~2mz<21A_nVJD~zM{6g{a9bV&1OQcf$TCrE&r5!s z+D7t-d@5~a1XI$%I)tqcR4XicnH>hwbJb9Znae=_`u~Icv%JJoMTbx?|6CV1Pk#th z0cva}#8v}4rygC3q+GTdtK9a(QhDIG#ys_6iroCkkM)ueadX|TE^Sz4(bvlET-46& zf4jF9%@tg_aUI{y)tw&zH4W=7oOeJy{BR~t@`UZ6YfApSy_Yx+cq53m(#5Cl_CtpH z>5Efx#<0p-`%4#zg56DRjgpu6|9z_bPZvsM;Ap$At}S!;o5w>i0E;;?ivs>;$G03^ z0d{Q2Bqqg;XRB8I?GQNu&}ktx$gXdi^e$kmjfCVWa^JMpm#%G0mF$6KcTWga?g`2AAY!Ea7oFGQ)QR?8BkOq?g)sWdflg$MPR`AYb&GA0hJ1u$#%=Bb#VY#*3 zTJPibLKvxXXaJ_tQ(&6zyFBB(Q5D+|EU*-$!sC0X+rTfsa$(rsF1+L`x2PepY1=Hm zANlcl%gOY_P@Rajd2P}HL!YpL44u-XIVHGJB~c-vWWitA?mw{VL^~dhOY;y6#FR`$XxMVc)aZ;Wy*n-fh^aTY~3SD zDLbo7M2#wL2pl%Oeu&r8P113O;X3{cNr72{)kSI*5FG!E?NA85(Iz@WJ( z#)9w(?+e>ESv-gxm9}kxfbQ={5>8PEeH$z1=B6NHd4|sKZjD<~uYkzzLjQjC@wQ~h zyWe($fbW-BOVr!EPblMZ83b|t=NH@kTcEAp4u+Lsqb9WLp^2&BnnWIO&8h; z7q#R@pC$PIUbo&HVt7F6bcwedV*oE|Gc+_^_ga_eEnDqDO<*e8{E-$X<7k(`$Y{we ztWS`aTyM{UkqnCBb5-{qJ~J?T1ZRBtkJ9L04;~sP{3|0`+q?YO+n^8ek^vpTKc(dV zVLz;sqM24&{LEHh2iMuQk{!1NQs*4Fhw=!n7E7)1`MDXtg>%5}>p(|(PtfMiibmyl z&9&;EWA%l$vyZ)Pmuc>0Wy@Y-6%7+zvF4n9jHL*CoAi%k*koRL-vuoY_;R|AwFXzY zk|G8@2-*S-|viZpJ{H!lc3?Og)L`$7-cy$8f3rmW3Gs ze(LX-4HbNn-g)omcE`oQdOBaoCEmSU9|~RdXbiVB(m>X_q>NUblnsW8#?Q6HZy(X+ zk>OR+=To`_uf^F5%^8s611%i6<-ny8Bq4UB!{@8FN+Wm2fLNX?(ZkkK+zI@zWp8UY zn(UpY_o*~V^q6sE@nLZGkpgP5f`-RDS#&8;tL6-c3b(>g=<;8^0Q^8UKbWphIvc_v z6x>o!nenJ{yB+|-$T0Sh!oQLfwi;BI6CW>%uC0gQn#>r}I;Dn)nq`p}8;bCiXK~Wb z7T2&d?-Tsc+LdyACZ6s;a7hXh;q!|LE&|)FpHi*u58KttY-iO4Wpzi5q6TQfC86^C z&T6X4HdD&2pJt$OnK)euc^|(OcS?JuH2!pV%GQ5v``+e*)B1q7pPR3ee0b}t{Otq? z1EaWJT-vs)zW%~LxB4QUe`Ey|_#b8XM;ZQYZT?Y){|l6%T9|hH%zw#*IC@L9Ol>=8 zlw>)9V>y4Q6Ta2491(mA@)J}d$~OrybFpYR*l+j+nkQJ|zPW%fY9%2_`;PmzYSz3iLx~s_|yu%3XB2Oi^$ATLm{S5{>}WG~o5Ktqv)-$%860(dy>e z&#F7!*nltNGo7CYQwXKz5_YhFY#y8PByGFEjl015p}sR3F__C7(Q&Mfn0a_YJ|_of1Ju> zULj)M`k@5{Tt&oJuGbdd@{2?1TE(YKL(x*Cr$42WUq<0VxnO5_Yi4RyTD@T&t*MF? z7&FzsfPaTMgD0a0t_qO~vd?=!QIn0dguy_oX6#OPO-;l{=Q;%|BPU^qjHJfpb5rwprA4Xs}rn;;<` zn8no;?2ASJK2;F@@WazaHX{^gn7y6A zCQ}O(<-_3}|JwTXH~eh%7b>-+4?n58a0~ApCbYW$>L?&=)Rmo1{SPDp-#u4|J++CB zz;}1X#Qy3fCW2X?qF2TLEhV!hQR6n<_XT}&{SJ^C&x%=V*gJ841&k}n|A{k2O-4(= zejQS}|T=bKi2AJo$|p}HtFjmLKuEXmaiTXwvBF_~ud z#}*%-Kcr8jA8rs*tSlN#D32bG)s~g*Hb;C={xCj9GY8aggf?o<4q$;phMHtC?n{nV z3EimnJ?Z43$KkgXZmKkrwG}uhD@kiJFWq!eji>U_M;rY`q9+YCA*Dj-aY=~c(Lf#- zOTZyj+?ekqV`vixAoSr|i?Zjv?}Ho4SE*fwAR*}|>_f{v;A5Y`EJ|jii_B7#?9rBMIgS2>S^W)(E&NXlk;~r3VRz`( zb*!dR!IUm>1w5{*{7ky-U}lpFmZ{lA+!!o4CYNg)TAm32HWIFVLEbzkTQtE=$aL;m z1pFz7`1!fnJS5#v01uD>&>a6Bh!^ei>O7Depb(&QqJDi+{RQ8R1L{3arWWid>ej)H zA~Dpf$;IPNVMNrLWli16@PQ{ci_z3_f`S&SqsIa>+(=(wr361Yz}elOqColcY@ly2 zXpQG1ncp*6se#4S#ISMvxvM@qip1{A=PH*iFBl_dd)>-fg+uP(=z8(!h^#k&vL`G{cDhd zrrAhQqkWqap|{5%ZJFc#)I`y7)OC#SYHAUa-(!sNeRN4MgF|K+Htl{vV zYxw49ZsTcl1%9N4x2%~Lop6d#uDlz`oDoMwwEuQfGIE0K2sRtrgJNs>sQ?= z2b$l(N5sU?sj7bq-OI#W4c9`|T*`9MvNoIm&w(pN)kxRa`wK|FcEeX7^L zCi%5`tg!Q#VOEnsA?id)#Py3y0}$H8K$SY&hlXqTZTErU%Bb+R$)rbEp`uX^WB#c{ zR(sjlZh;4x$22NV*EnjV$ce71ydz&*UbqQwZAyg7P_d;#@}MaY37t9>8gE}%eZA_v zr3D*TS#0tKFRjMb%J@U6<>H!!;Rdf%BBs?3YV+xn-KpIY3U$QUfy`7ywQrWC^a248k6^XX6o6O zJ3)%Ox#w`fs&6+)LCELY!mW2wPtTkEj0+=2n&tm7n=U9VXt!$@^Nv88w{jgNOJge8KntXIU&4-I3;Vrsv8C#@ zTO+St)WzbnrL0?%Y(x2Wj;@10^@)7G`&e+I$+C4$%}sGOHM2<> z+@^nRb4Oeu<}odE>%04wYrRV9vY=%5{^s~5;eA$EF?`oJWO#uaY zGUATEeBnUL0tFw1c*OiPkA4!1qZZ|B9yrMS>{zc6iDz0kM;fel=xcop&TTI>!TqMY z<~7Z!t<}^wT{%ueGzZJhopQ9t{pPE3I+d~-V=n`3^QC7glDKPTid8~S`}rw*juEn~ zBAOUWyvctiUg<`aYN{>Td*EeA1J|1hd2Wz1GcIIMXrfTi05jtR{hd z9>v>abHjoH@tOK{v8VrP#{&UVpT)_03own;S3-!VF;i8gQZj8Tf=Pwwx!ui7`3por zCt^49iT;xBn{(}R>0Tw>ZwBaw#y98bz7OleuPNZ?g?dUmSKg#EYfO!OJxWE53ZU6V zcT~7i^SyDgc6OnX$&5=<3X?XF(#~agRSLHwoTIIxk5}Ltc{$$iaB!@d&yQ7sy%|<> zywVRc9av=T4LE$>c)YO;qj)z{KNN{q>^9{PGC5&7c|lemkZ=0s07yHMf;>7{ov%~k zyKHIfv#955-q%+;bq1`&-HFtBM zuOh>*DYtN^QrxU@atyb#@S%s!{X&W%?;ZzQ>m~?0JyM(tmsG&h@!d!u;E%jBunHdX@c0&L2r?PwsTBL$Xtz7|ArB67qxxOC30$NrxCGa+tNB z61x3MDWAgJX`?9v1N$)Ot8nnm1LMZjh?B9W2AvYAB?5c-m#%cLw)<<+@}Qr;|_pKtIAw z{nZ;HMCD96?^CGCnN*6BT^L16b~+)fab8$XU3UpYijpb20 zHx~lnerT72jnj13^vgW)3Uih9n(VR^Zd-Uaw+j21#X^x@CARa2Ka`$F;f%?K6F<_e zZ@gYa@MlVtjSyZmK_}Z2M2+m=Bd|=p>d{O+T)4Qhv6zCcGu@i(y!XYCHnZ2ycrQldXHy z8uWt5`dlfJbY_06tz|OD7>!yUjOy|Oc)B%%T0PH0c!``J$7#;kuNAUSHt9z!=ORj+ z&?b9O-^|cQq$6QZSxGdQyz97@^QRqTi}7TSkO1c1rrrogPMSt?(x7W;pN26x{!lWY z7xLrfVBG9vRuJxeLVM>O^hDri_AAPy89~+%2qr{Q!gkGn2A>6+u0j(asi0xcu*xQSkQ31bPo9Vt&C=4@Z#|e|npMMzVk+JM; zFNfzj2J7a{`l2A&PWBq=VG#+8#O8X@u5kKYdD)Irj@T6qtH#7Ka_-wD_8JUh-+LK% zqhp$~yj%@$$+m4^p=bI(J;X$6g(2UVdL}Qj6K9dYExJ#OmLe@cFwBPVXZ(?jrwSm; zwpcEaG`7@4Gw0$z%g8*1Sj39@F6FIkIFeeHU1b-KS(PW@WUM8FInC*5 zP4brov2Dq??d!$hT;crXgy=O(?*ycN47GfUpWHy!!p&f0{MJex)CN0qQ0&;@Xdrjg zX;TDQV)5zSvWWIkieVTC8RQ%vuFbXJ(0CAHj0M`Xz8J1(D_fFF`LDIGR&cgXOZ; z=C9T3Z!eQ&{f+0BSbo<|)Xo z6$O}pHr<+fs*OyFQ=@|+@mIaSe%%5(;=9^`NjEXSgs7Z?}F{ zXjx1=V&%;l^u*i(`zv+hvTGf-fBf;Pto}~or&HG)MjdYkyWb_k?II$#@;CLOTFWrJ6q8h)N?A2jdH*S=Cw=f9s|8++=x$vZLk zzE0(?do`3|AIY6+`_FW@zd+mE3_;c!U9mS%@sZF5DwuumW<<<7U{MZCh`oKt-DE?l zk5!uo>5_;6$uiVKZfC*&!zJ1&21Od&*FJ08hf7xI&O0oS?u=p2DbOL-r z6!dUt9E6k_+>HKq@SbJ@f_InEv)hwNTj2_DTcsMnFL0DH&3yIM2iviLm+TU`f+X_x z$bb8oBQ{ciI!8t3iog8ogTJP~2?+$a0FqiuwClIg0;)2g4h*_QKivkAeSKe@0>Fx^ zatZ$?XEALMurbOL3p%zlH@8g+AQ#OiCyZtSsFcbuseucJ6eeaXX&d#8w3&^2-3ubA z9v*qi?FYG%LFK*)FDO4Pq+{e2xoc?Uo(t3MVjJ{7{G6v8dm4Xxa)66kPvJ9e*kEuU z%LVE2jrmk`B-1HWHHzPw!TlE;p&si*AJL6ytPn=MZKvDbaCOhqD*>@2K+L#^yBCynKUpS2)?!-r6Q)qQlW<@`Bs)fz|v2j^lJHtCt&5V z^xWkCD1l6Z(mj4<#uAlNN@L|l%v*KYr6-)Jg%lD$Yn}!o6c6mYcu^HyIUUmcQdRYu zYpV2^D}Mu7TaJTP&oYBgMlqUdy}fY)s5?T8(E9Z*X0_{c zA@rfgtWZGYzPPqH#XF85=h-+Jqm=oyyIB)^ME%AVyqJqU#hOnVCf`N}qS?3*%Xci> zi;V4o0N4Mv3A9lgvGKwBpqBKKdeM}y?=PTY*W&a0-3QIvT(@G`dZkLP^aQpsXn(A~ zMf2Sr1mtLiHVt2-e@s8zco+d!H}UNfTC84;Ky3J-#>s`Xm-$fgqAA`p6zSCrvEmD) z>*Idc`^VSpVG-JJI9fTV{pMi>ISyf?6X?`+LPE7ipL2x+6Q9~AqP}vnT(fd4rq^OV zKgx0OcsjNVy!>*XYieX)CAODJrjG3I){V@S#pg;hmT9~YWmrsEQI#VlI>XpORI;FQ z(wI;;SnU}>BKhgBug7{?;|?eu8en}gD7A5hueL@Nzve2eIqyMqUKSMXOM2It9R+#s zu8)?P%HvAybqf61eOtM~Ck24cV>o|pOr%2I+FjfN@9hok$x#aH2`CykDr(=IH%<99K%h}pH|vC?mA*qbvsY3AcR*lM3hItz8f!N9HcLAe zv{DOJuWf}o1IpR?1n!bxseX&n z?5O`(p8z$^H_sX-iurEocDjUL?mmt7r@xF7gLS6F$M zP_Hw?6vAAT47IQK^u9lU@~!ZKwfRca2qycKyJF{&_rW>Wsh}Ah-O(iL#I(Bh#HoHH zr@fJ$S(s!{n1C!d^(1$_*hm$AyyuWsfxqbq(pgs6@SwrJ*>%cdADX^?H~;w z*sw{-_k8!;sxxNq@?9bHx!i_rB(#wWI$F~PZK90{0z`+iA;D51VH~6X+4bL-mMP80 zSC{)Q?ti0*(zZ$Xv_42f48D;FMs*->q?hsK@_0kIzcL z$Yd+YvJsP4_Cib1*r57iP{&0KqA>-+aDD&F z(w!|I4x|86VJV(5n_26e!Iy?mdSLex}Rd_-p{4F1|0}}fC5M1M#+`^+E9STo6(j#``A;7p98G&=&a_zrw?ngDCou7hY+D?~PH$azUFIit)c3^@w#yqJ5_N2V< zWnv!=S{`J@)VKsQCY=dd4Hzv4W_;sDBH3M`%4W z^&p;Sn_fXtpV=LOw<0dsCTy^xc!~FVg;e>7(Hix>?r?Y_mHhT*L|ZcymvU0y4CNL~ zU@l#Mn;`=@KDRWJDQbgDv3*U94SG+nV-IQ=oPW4B_q9-p=;z0bG|39;(1;xYE5$O1 zIps&m96z6|;A!F_titWAQOlU=bny9Hs#(FBi0O5}3k|Qk)<>H>>AQ~9N8yDN`ZL?Fi?9?MR<~!ZVWu>FT*x{l#UL}5Ej`{z|TIy)-Oy2Xu zStCW`-l6HIKNw%PZjI|sD}^Y6+^h;aHFx26w`zhJ%BMk=R;vqRS!Jga11>7q8X#oc zoml?6h6imI`(NDGcQF1S)8V)GkU=&^rrMo)Mjo^Zi5G)r)N~mKpYH6(9N%(c=BTBa zA46>Rws1WLlN__Xa>%~6rz6s$sJpoHPTtH|Ggr5{AW~!MQhEMEZc5!SR`1;wUqnfL zC!5zVn zqRLc`;r3yCyF1QFg#Cy&D9*f9ONe|<-9CXM1~FbGk2F4@wRY_}WZLoe?dN#~=9)4t zMIKif#?N>K|hJ-!`$l!Ijv9%HJ2V8PpRACAMwT z?!W0oDZBV;sYADKdm?-L1AkQ_fGErgfBpI2^k_@^8@T=FMElHq)MU4nmu?xcsWF#~ z)9f#+EKz!-e3qt9Uu#?`q5+StdCs!UQoT(P#XCe5)#zzwq~Lq zENMBvU8REeWr$i{uCO=?<2bIj<-7%A)!g?ID3iN`QMUyM-vhSw*W1Dy`Evn2nE`HP zzq+<9|6jZCe_R8*_gME{hs(U-?@A8RIW_t!Dep5gcOxyxGJvx;b|=B)ec zE8#V-h5E`{54vowlH2Wo0`9rIWO4S|>D4No`N-|uJ8scE9q&9_>1X6X`(xuVX!gt3 z@JE0^*EB#s8T5R0k=J|QD7=!0!^x!JFpS;97Xae)KPyKn6@G_HPURX*0t{Ru0215vNQXgcRB(*e%e(2>-#&+)S@7nIbZ(F zKj}4>&Ht0;G2{&kb@jEdE`h>I1@&O9_&E?Qr6@LnG=$YYbV-|>B1lf*@1ATa^~)8i z2tqBb=uMZ6XT?$~t7TzTjlI2o6)1}*Cw&^yBjnE>R#?CZN)NzG@`nMie%G_-^%U^6 z#rr+ptEPq>*IG4cXF6=%J4(JBD~m9H9@cWKqIlwo{LXn}PVgcp2Q z4lB&gvR!*LyA$SuM#1pFxbEJDa$|&?cNl>ldtsJ=XEA>~G=MOfF2>IonzCCFB)N7= z4Znx^3;CMan6Dk|9XIphE4aNAwK^XKyMkCt>-9*~ZaA*JE?eF^48cSrwXJtp`FwK4 zDRPbTk=|;?K0Yk>)1m7Vv!~>JY~PC|yHI{7YlU>_8w+&&PUGEJs99n7Auvec>V?SM z;&bBiGo89^p^E&DjKRVB;6;Vv8*uCT$B@o9U68qv3UY#dJr|pyQ=Pai$f4eS(D|_o zbeGNScwfvW*68{o_>}qdMHN6b%rNLes)}*Ov3sr!yi>VY;Yhd?vYR!FCN+RcZO2!W zBf%dZI$InI%$IZU0*nFHaG)nK=E4qhY&mC0yWJrti;Jl0xuIQ6jrkR%I7iK_RZU0} zGA72NnL%UuvV+=*{tDRoGD7_68{d$?OrtQCyI6)}UcL055*&vd2#rxBqvoA!&Md345Pm{Y4coYIl$YC<2RHFl9j)C-V)!ihg!51Z#|YA3Y$f5B8Gj{sY~OYa`rVNsge zz30%W6JYJNepq{8H?VM}S>A&rn<(i4I7iMO->F;(nDuuadgY65;tIo<&1S}$=Pkb{ zp@t&`@2r3@uUkMn)u9+Oruti%=Yra?G+PU|e2E^c84Pq3Gz8^RG>vCRRnJvEc=y|d zqXda=#>5_a5Y;0%zQmgE#zcp2Lx#RD64UN>eMpQvJnO6iouYJ>+kT4UaUFpk4sGXx zMRp~pXjivtBDvHGk7yoGo618uo%LQ3RsEN*`uYUA!kU%E$XMn=tv==D*MI4OE4ZLJ z)&J;2yP@$611(?F_ol03u-FRdi$%k;I|RRbB6SmU6CIiAW8!Wfo1Lj$Qs|?p@|Td> zQe95(TF!4dbE)bNKm+CQHq|*{(D?Jp!A&-Z zs__Vk#V%7-7%h@0W5sS~s2a+%bYnmo5RD8qjC$)knrcQ3o6~P`W+Iw~mlR%cz48KmB;)FV?lP4Bh-t}KF7uSM? zMe(HvUcmp5rGBOmDe915=`@pMf8|d+*oEV2v{PGJ4jW4kC_mc!kda&xj8hm_r)M)g zFf-ZlTCZf`(f3L+9#qpoU0&|;@Zt5Ty2>v-h($hG8Lln!#$CL{DP(j4G&%0My=z*0 zh(`hbi^)25ei#ZGH|Nf<2sMYLuQ$&*4{7%ujeO1%dF0%$;t)|371w{vqW!V!w*ha>RH~`RGU%y2 zNbs~K`i(Ob2sO(o$(||dGwCimkt-UpyU=GNG=TgVS<6YDU&>@L=Tf`qdQQ8FPa8O>zNB zPAJ?j5_Tp_|J6+0a`=mK5$+RNQ#}_RufS5hd?Y`vlT&BHO_zIxS3Twt#W&_Yn=4+C z?cP?9nPd8(bz?6y>$z5bB2i@_9w8qc`a(g=dYs@^l;ttq&Tp;SUcp~`5>i*~JJMuG ziG75pk2<51BEw#F6!oPrrjty9)~52>J{*?K89F6I;>w~XT(ffVkg)WLXmc}ERGmhW zE9uy}UgOrdxhI?{AtKZ|&6_nHmSDZOPcy<-U9Kr4C1NvPhC^M|*BVI+a@sPZ#sp3$ z{D_T=>|LKH(1a736}&FJbr^Y_`WM{Ajn=6+?gaTPn&(_Lx*^$<$tPSjUF+~ef`0&D z`4xmF!dBd;>7#4(N7eItxE#)kLEA$6X)9Ke;r^2p{$lzJV5b~i9%KbdPwgdPCX=p> zq8xr;jNl_qN`$u^n)~7^H(weGZog)c6U_Fv`m~1@rOea|3QWY>6z1Q1iN^acb*BzK z|Dj;UcJ8UW{A5C{@Zv2W_fZwhy<`gWZbx=N$9Fa|gVYk7Jl-ODDo_3j*KFqF{$1!! zbBVsuDES^U`@yG>jkyX|tft6I0qLcN^FJ|{=>EX%xo9D!QDC82I4j3wkiyX^Uea1} zt8dtPPYwp%(--#lxlwuw5i_&5Qb4_4QgnWd3Dbx21d@(mt@-J%S^lf;kABO5IRsKt9`^B^=4-I~l2(&1PhImOJT(mKc<;$q zmz)gsDJVp3*um|hyDS)>5Ai}U*3A$!hlY=jloT&6gj7(vk|=uhRaVoLEf(ujg_a_` zJ|n#JuA582h1_e{UWX_l$j~1*oF{%%v8)KWzPLahUQ0EJ9!MAqxLw}~2gVoLkKctC z5vwQ_oJqo03vT-DY4YEhi1zX1OtPbvfZ_v5@IGh4LJ6QlglQvus4Lxf_!}b%tAu=0 zd(3%LKHnX*ien&RLDhT>b^l&UdEJNsqj0S2|dc*$%P6VG8qA(u(n~CG&GL zFQQ1A)fxEH`K2Ar1eUzd!foW|eP$Z@7BSHCfgsYxX5qdz#wAu|DuDNXJ-1x$fYmI-|RI`89|Jvw_ zZ5~V6xkvpt#F=_PjIdCSoK`P1(})UJtRHQ=kX`lOLc37kMX}ZGG|R}ZSj3by`*<0j zHLcdn<1SykC`E*y_tiQcNa>1Xt&E!+%PXr3+|C+dc2O&=^`C{CA<_lo<6~HJJr&j! z+NwSH%rZ#xf_hcHDN-{rfjg29;fn7=Rs>?g9^#F=GstK&KMpOiB;N?Y2eBge0ESXi zW^8J#i6*15a%L^J7)y!H0_~3jsUFQu*nY?P#)*VbZBnYdfTvdoE?<6}L*UMrFj4Ku zqeh#3Hiy_9Mgqc5-p&%2nXegdDy8T`a3E8UTt9tsJjwkIQ(iCfGBg!F+0I5SqkbP! zBltl(c9>S5bsuqVKVKER0DegCI7RPvSZqYp<*=Vl9~Y^AA515{RBabKePk>~+_nlw zgbaTk;VMF>E`+UgwAG-G2Kot9AzC<#Z-AD{;ztvNS*x^pf~$=5AhgwqAQgiD#VX+s zKZUBkg}s=21=v_}gscQZl4gpLiY{c*qebb-`{=92E)uqN7rEbNEjz0oFXl9Laasr7Zx8bRYJnS8Kh|Yy%&nL6hbSi265l|S zs&~IFn0!G7r(V|pP==}=9!8^xzq|T7>n#0yOEZJ~|IF^@NiN?TkHKMoW=8?a*3<)~|~*wOwO3)vxMXvM0#vsf6{NTe@BF zKr^=A-(R=muoSxI(5m(H#whBHKHk#Ld3?YUD)J?iIn7SbKg+(tmWd5-K$YuwLspN8 zY+|2>&NZ9l*|vVdui~1&An`=x@)2e;aqlla^)LIK>g1au5EAzdt-3&UxK>ai5og(T z?>?-f5}JzOlO(z2j?^WQ!t~9&MZ1kX-99X-(lmqbq)UbZl;GTblZTcbhZEc&YgB^U zDv2M@nP1m;=Eb3Ag!t0%ba0#7l~=~Fb@z1!0csJ})uV<)Q8}}cJRCeU<^x%bdPOUe z1JP=HFw(cxAr9W}Z6tY6ubT8BLN2bWsBtV6hm?^a*C>X%_neNtcPI9O!n{y~{Kg}P zY7B>Sb45MqWU?%V(W~AS31byR4w6T+Feh|4wkbr7l1!5-ti*-VNFJu6c`?W0Dc3b` zBg3#l?wDw>)x6witd`*D`=rIE6q(5iSmlHf~3Anp&i zFT+Yi8fS|7^)0Ynirk=G+*!Xh!6?&*34CJ4vdS$IL$USM5^XEQ!zHmS_(Xt-^9R?2 z-!-B8?67gyu1RAP(eGB0x?{W`&G3AIcNOO9gX9O4nb;P1l( zZPZ8nvnVKSycLbsvwBju{q7ekKSiR^F`d$oD&rJet-E_Q(I>M@7y zbC069RFFKfR?fxGOz30Mmdd1>prrfEZs$N1nM@~{A@!(d>@MS~mqvyFie6IBZPECl-V(@CXBWQ_80u1T2QBnehccyg1DS&`8rO= zxhzU5#D1uAeoz~)Avz#%+CIzt0*{QFfkxb{hlxkyZURcHP!-v&NkjM5BUrr;7L-9b z%|f2x)*glSh`6Xaz}|Em|kLGHfc&cfY? z?0a7l3r6xU6vejbbaq4%rRK2}Boj;D%aGETNu2)u=D2I-rncTa1%noiqbf}+J?6H~ z6>j?no#d+CH-9jI<27W zhM*cc_GhE^&27bo-7g+EA;wN|#ebFJD&5%_DmDK(PE9H`blW+#4G?E5qaB+vWnJA7 z^tVs8I;qdKL94WXZuO6>*qZl`GW?&vangM24o2Uv`#cc_p`n&SPR8(_#;r55D_3Kt zAj7i91@hzjZf?u#-Q#;uYiPxdtopa@0T-9le0GpEUwjGSzu1?6w(~kC^+Ur6w4pfgi&pUYxSL!)*8Pt4y>GWu zTE6@KC-@LrDDKYcS z^{0QHz?*aEx~y|DsJ*GQ-vMc`}6M+iZtu#t*& zb^f^#1~shh8eX!jl72N@Q6fmpiIex{NY@)s5Dam(vn>SZDmw49lVV4trn*&AqENP; zay%|zZgH1-%;@n;FindRc#5E5jCLAaBB*7~5Gg?yOyzIs5X$wdH3Wj$VD4==$!X4o zM`{P*>pV>a2TK$vtL;gLAsg%7Aq!?}dw=~I@!hGqf!Vm(rQdIb>*^%@3)eAVBol0< zekC)yNDwPTx+6D_DXmMgn!?U;zq^*IG1X#NPR#psRPF$!)QPL5+nZ1$zJrYI7?7E~ zl)7s)${JG)-#MDE>JWpAy4W(e?`0vMv!MdJ-QVcECAhH5&AxIhoiW4gGRT4Ardj4h z56{PR`Q;6Tg1V)-zc0J}ccD_o=~m|3eDeyWdl!@ot2TmfvnsZqdjT%{Pr+hzbxpUb z`5T9Tg6KiH{BIJ5e^u%SF;KtLn{W3(yJ7;)+xN1!=eKI{)(x=$b7bo)FHQ3t+s0txNjd2M4!h^iNAfY zb^bb^flN;lZzO!1asQ8UeNmME70TrX^F7(vb=3~lUPMwb4@1bU^i(oZNdFIe-xby5 zwzj+M6+6ADG!X>p(y;&{O$9>-1tIiE4IR-Xy(m?>O7B7-)PPC}9qA7mw60|T4aYgMCbdP2>aH*_s^i%J;RW5kgEBmhk6C|4!uChGW$!0d# zz+Dkq7;BbQpmdV%26LR#bgrjsav5H?qqDPG0~*7~=2f~ z`HAFi0I{#d;20EDa~IXxDX{sE-eJ!t0O(dfwm1joHV#R&&U*U`9KE2qL-1w{Jv7GE zG=x~5-_FpQrB!%iMHGz&28@q}7bB(Ue7+e-khli06uO`4{pADtxq08&9HoNU2fEpZ z@7V2Uu0%+c4jR~YWl~!Wu}lD}DpYQd;Aw_@^Ny8NzYKTV+uGkNIrM=WygpFFvdWuz zpNJ#?gFmD!x#R#TA6{faKu_;{+{Lf!q!H=GiyG`2ku+&UStkAc#f)snzIs#hrJ(O| zWYJOTJ$rPLR)iBI3y?Fm#A5DuR>24Bx}5P-*!xDiT(k#|j-J0h(8#u%lO$I59X6%S zf@}=mB9in3pRH4hjx8LdbjB6CH%5zcm)PmNt<*`+mPHQ61da30!v1%U{BSNoPZUtE zj^=s|_fEtqWz)q*V_~*Ed6mP1EkuqI_&meQ!zX>~RM^mm6NKuWCJTgg&2m#VmT|NV z7mA{6Yu{bE2C!TB_r)a@7ua5C2v@zwVW1JERi5B59%AAu+|U(0Ci=oqBC;z2VUE&x z&*VVSW`Y$A&ss7q!#8Rm47&WO1N=mYB@0G&p3g~&0qx$3LGhjQ33hgNxf64QY8-r% zcf4`^x=fak2(r%I3&!`$G?%(6o*SM@5SLrbB!T$cO;W1;! zQ%@p~B>$-&&hn1@#-GrAF%V1r-h0f^gT3aW4DdwR!LSg(G5i@&5K@T@4FmuYv;Z=V+_>y-jh}GJO62G?QtspUSWu{ z=q}oWf818#&!1OKo|D4w+RwXAGtwPun+dL0w3D;Y&R6{RS^rUk`|o`H6psJO*U!!L ze^(|*^kK5#WNwzk=k*w-aKkf$;k(p(k0^@k1DR8EcGf$u*4l$5YA*9vrFOTwiK%*!VlK@9#Hx zG;@LjDbjhcc!&P;wfA89uy3mdDEfF65rgU7s9E2YOD!dOf(nM@`#8Nbt@QrQ{g2*I zWnJV}OuOaLR3R)PW0|L|;ACaGIW_AJnWZ*{u9tWku5-3WrGG}y^r>G9?`%_YjayZ9 zqzE6gYV3SS7T5W;rb9*4=D}FxAAqCbyT+q|fwaRdrp5++9DN$)JwMD&A4ON)atWx0 zg#5OfNW5XevZJGOvVrLga=Fd5nR6L}ikLdj=(_ymy^e@2#*N@2u!XPKt_o2Whb$7f z5RJURDUlf*1b+Ck2BXVY5y2pzED0wj0YIjft(Fy8rR|^ z{OU3Dox>Uf2(EIZeogec=?w4VZ(k3s3nq^>oMNPCPf@;>FYjeZAi}owK=g^;ib!U# zR1S+TcQuHVK+z)cdIien&w8fGaa)tcj1Tpg(z1oc}ALEISc7z)TP<${wvOuzwhn^pymGq+fNG`>}> zV4>=xGDQP}K*XqFNR*^YG7@`T?Lb4!9mlu~k|e)f^|TLXW7P<6d1L5!Eu*>}`oNKZ z(iCfO?yByFfQW&f#Ur6G%kuLkijFMqBl!Xo57-HdoO;S1fk1mKb}UqREf zSYPSt{`{YQ4WPvA?V1X2?#`~?DJnk{BvTR|JY+9C_}6M6_Q1Ud9(;!{&_1&Es)@On z16t|sRTD!FJOG+|*2L(7R%LtD#Bc!HzMw6nc{kYkPV*n=2Oedhm7~e7)Z^WXBx!F! zKs|t25?kz%UmO;@6%0q+#Q*bZO3jT8rDm1>H45o04NFe4hPSMnZNrSF^~-rn*t2#KKxgP|a1292&}4_@gCI z!M|$*n%=(%Ao$nQK>ZHoFJIA|;EXCZFDs~B;om1>J88X8IylJCJ$PNPOC8s4RVQr# z;0WQ~gt_h8A)20Kv+{R-fD#5ftCb*2?-Ong(wW6In!Vw->^Oz=?*#u58_y!$9vkmAzA*ciJc2P& z-1(8hB|&eZ_XDp5Rm@1D^k`$0;H#JEe~El1Co1j2`{Ab=M}D4D7--ac~K6HoTsV zdlxd0QIUsOD0antV;Ujh6_Z%{imh!e?JHWJG@SCgWtWH#mD^_Ie|%pdPDHp3FBj20 z;E-k2NjBpOgwyaiAnJSi^|hyKR=&=2D2B{N;A5j){@?n{Cq%ByPCdpV))2E9#n^-> zzB`qp5pF4D(>426z(8h-T=nZ4(nLj@l=W&SZs-x{uIMr5{q}~(=JlbY!J;+CiStQp zet!vY24aWbdog(5KD{o(x`B7!6quf#)^EY${YZJajgd=F{54^=<^`QdtKmA?>IuO< zekc6&c{r9>dPP+hZ;2iL#N$xHET8_vk3dr9*z#>t z^UfV|2tV5Wq0ckNQzJk9I%C3r8Zx{+=c^I^9P;f<+@WWh#Xg1{pG}|P ze~3KI>~{*Kx(TThS6C==InQRls#HflZ}YqyXYt9*{z`+t}1!KxTV1xg+){d?!%^%z%f zWQgU^Im=bOSYK$Lai`GR?t4Q5TtPDhNxnrnmy3{mK^ZXO1KO${4p{BL#I%)Kes1-t z-BqflXKIBY`~#Vu1MM<9MM*VKDq`OOFRm*cF(BE+Z>l|oqA#fqd6o&{ee68z)ArGE z!H&)q?a}FG54m%8eg0E*ZcEms0%C=#vL!qywyEz!l_Ob&1I9gtp3q$G&>g!B5gmSa2ayi;VrybKG1bBpk-Ri zb*PfG%F=q?Fw2p8t$UiuDwBeG$qLbJWvN4U@^*=|?asmDn676Zce8`0%6Wv>ysU{9 z=pN8kD$9GQWKeWlKT0oo3BgoGD zeEhCMWe&*VN4_AMF{=3lhvLBtRK@Up56J)R(0;x^l~mJgh!5)eZDj5KVK5QA7t>$-_JyRH=B(MR zR*~PXb^S#!5r^+Q`nN;-nYVDCRMQ-WmDK!wWbMDg_18N7-xaPBHTxwBq$LTyCLvJl z(#EO|>wpuX=?1{&HR#JvI|2Xr3!&z{GuH<=_cdMGDQT&l-SKsq_|D2QEhK<@XUPw0f`s@f;l7?RiE13U+@GF zAaz3kQ=qg1FJ#|-KajdXp3o3L^@dq$0tz^=PR5b6raKUAx6#XNi9coU?>%{e1Ej2E zh7xz45ci2BhZU|fL(N4DZuxTcKS!eSA)g!&1^TNy?-!|+)xH{r#=k2FJE?oe^;02S?`nMsaz+pKCK{smFjQZM34&g;P7%EZBD__ z&L8V~%dMfz&KztcE42hVc;%-x59cv59H8rO&xC@BEcygO@mmd9=B=9d@9=FU*-gWp%34q8HO%!c7##{=G{lDT;#oaaplL0!ywJh?J!veybk> z?`CI*A;4WYaa_Qucc@Z-V>#`WeeSM8a=fG?m)`B^|K4r}u*%ltLl!lCn(lqf3@cJNkcbj%FX`5T5=oQ;!7Yt%7M3gG z{Q0@qlp{keP4m2)UeT?)9VR!^;K=QbvnpOsjt}A?Fwz0zH^^O*V7rJ|JHq3>RrV_O zdtFqK_T6A^F24fOgQfduqG@EuO-ULZ%aN zQa6%%%;}iAFv{<;tfG;egHY3wCI`L`3F!HPMd7cyxh7>AUZS>HPLIM2 zm%YF0Bs|k}LsW-Fz3ZMiGQ}|0fY~Z?iz8cak!Y!!X_!@InBLme!#}XeoO-n?FI?Z| zz4?SrMo-mb5Lc-nM8mNl6RPX(>T>%uYNL+>PzTt^;~B8}XdK+fv1#jQ)PTdv&)9^H zp4)m-|E1og@Msu4TkINE{kprG*!C9$4}7kl71K@$V%1h%)D!3DPWU92sp7loJx*)JMeWTS>n3&CLw!F>fiNEB!>KSjMm1gRGBmeH+eMoKEV)&rZ@z#a0nc~_>b6}*f*qid zpp2j`ccFt1pFQNR)Qy)RCiG;&uXq_=t2P>;${ldqZ$Uh%-rwRh(zH=NQ@h}t<#H+? zYEAsa3~npN(SQmG6nwH~j7_YIQVx>4@f_+BG+B)f!1qIAKi&Z%9pMzCz? zoNn5|7Jz4VdB7EZYw<`W%xHYdt8MZDts!scXm4CNe$ZXHfA9(J#>h{&%N`}#e;|1MGCR2ep4#5A8}lc z?q1sVwXZz~)rev_+bGnJ3Dr2`Xbks2x*Y~N4!941%OI@9V>KUGmO&WnO`SCL22Gy^ z)ogKtb_aw@t_EpW{k=#P?c7_ig&*$>AK^pL88DN!>2`ayf&6}Q=b+Z*`i7J^$Fd|r zA$Iv33fI{M*}ib%_Q?wuA84dic8tOEsi{7s_pPestYPKikH?wWqEa*JidN4qsr^v z_svRFMNkgCl~nizVn+%jz^+S^J5mOZ^3RA8!?fpF%_drv0P^T+3&~P;j_m55%!8!y z@!#!h=R?uJ0H#k}=o^E}M0~W;c|O;glerpSTHfhjGh-3N@NpeE%@E9T#|9w*Xjg9o zGJ)5#Cln<3wD6@5a>EPGMU1UA(uvOH^;qP9ir>%6PpZ7h^}8B3TB>hV-1cpf{#$;@ zZW<=(9O_$|2w5=N$~J5HxjO{ss6lXwCaG%dOTx@{O(<+u7i_&am~)L$u0i6h3WN31 zhpwum4M{qKHD;;gw7GtrehtRxG15_*xot9(s}IP2lFe3oEaFqO(z%KYC&PN@l*Lwz zXV=5%DV-;$%`zi#h>huZrQlfQstJiwM}JMYE#0&4ajivxI`+NSBrN%>=c<&G`Eb}2 zIA>THOeE>4Wu%~m=Au`RZxi1{l*adBl(-FS3lTkk_Z*-Ta0`Aaybzjz7}i|k?Bs=p zN-(Ym7=%zole@2D;`4@^X`*o!k$m$f!KlpA;tA%8wq?!n(}z3k{pg_^J$@vXh)BL}jKG+*8b84b2y#lQd~y;~$<5 z**I&B4*-00r4L)h9X#N=lwP`%q&e=Ch^R7}b1_>lPUmxU#^u7w7VG8fP}Qu+^dNwF z5rcE@$n~l|&3%I}TWQNX3@|w<9qDz56oTv8nuX44lhV2zn`Lc+oug!yCv#OR=j`f` z%ckk3x_uZvsLdC-L4T`m-MIczJNp=T^O%NJmSebSB;UOeE8M7WpXBBczW4)cODww} z2E%&136+#EFU)v>SnrN@#E`nu_ZCmY(v70qj<541RKvSV2UZ6;=lT@U%LpEOQ8M9K`mfRF5JdsFQ+ z9*8>`e;}#GsJYgGkL`hz_PD8fjG<-4kEe&zL{|sC*_;dYOWkY%{WJZme_%1r{j(ny z)J#52+IKo73#)n6RDP^+*P=})t4v|P%gk(g|kJz+sfVPb2xeGfp zh!vUWFUP{!Wbkv?zi9@pmu$Jc%|R__D#S|&gwvOqL*gVz)#$?s!i<7t*~qZK@BJ?A zE$Dp&Mw;^>1pnlS_(1}?^assAs29NMr)bdbr;F@AAcTUoW=U4?sCA4M1EG*rw+{ze zR_e7ljN210&ciL6pW(wK9jxDBNjaF~gA;nOA?KoC^ij?&>D+KIYnXEpX{&N|jR&Uu zPamN#QErUk5s&=fT7NnA8=uYDV2TsIxwD*Bk^9NdM^8b&2}5?bKtlfJse;@}BLP5We^SluQmR&QILyUbgb6{jl5I@pJ|cp%lsooXBqx*wQ0Mdh`V zKUgeq+WU{fQ~E20NwaPbkRQRn>#P0WR2gnLn3p9{H8g{M$@Ap)cJT!FXj6Gu6wE5* z7T?c8cBc#SIDPfd^TQ7~3&pw4O?fg*llTTYYj4VZ$ohWr^OUPB38Ye%ZYyinEn!e> zcx`R6GAOTvrh5SxVZoiQAaM`y4Sd8Z^>!o!>+5ZQ_wo=a^S+0q?^d?bqU$+auhs@$Juyu`d$Y9f)u%>*RMXti_WMI%o4mcv# zCMr)e3);|rNad^fxfW^H;Y{GhsD}C?qnCO^#SnT zdoio15&fw+z&ilCO7Tix$vnA4aXTQqGLNs?2=xxVN}=8lm3ogC8qiivVG(p^(58x~f6}Wl%=^f~5BjgO-<;B{S3@6h4d`PuurBnYjh?eWR=E5YIMGz#G;)ng zr3(k!pK>cB2UqZgy6X51{Gk&ioV?OFP~}N}+lworA3fE^dad8MQZbm|Pk*6>8BgZA zCD|g_+vO64VccGxzOhkz-e5LNs47!~ZzM%XF9bJO7HE4fAFz9~V7O5)=Gz)tT2={7 zF!wjEh!Nn8H_BBpPN_8`ALOhSsMuxN8n3B;5*{eod?14XQ#YwA=>b22t2$tVf<^uA#q7(sXKKJvcwJph&?W zRkh_>VGk=ISbIadm{ho-`3Tt`(&%etXWj7Jg@;XWCYDAtv@(yvU@7GFVn_4-B$Dof z?B|hhLv%?T>1$|nKuFj(EWnfWInAzdsi0Q9oAl9lr=YO%!CasbNg#oBQ_y1`1Pq)2 zp_!3%y7x11@5qU2Jz_9T`hcQr*b^7QhohFToith}86+ zo_3OPl7{BWJL$Fh%=4pUO_oI3IPQC$W@MNWa*yVoD%^7B>J(*Fj_hzzQ7p5x%q1iI27Gq8d>4;h06&A9cMcfv4<2LR>D}7Z?uXM%$_6+F#xLvc ziq-wv?7j`jG0>f)w2Q-Yrzer*;`A5?t$vMP{_2T%H%`j4$d->i^XoUeoEuHRd;SlP z$1g9GYB~n;EVLs;F70sQ?DPcwYIXO`F zjzTX;6aEr-403*K-p!!_QH075PlIN>F8ob9YN%#r*yG4HonD_Ker-6gw5ClwBJc>K z(?!dY^}aQSxqj)@!DlZ)WuQ1dXe@$}8h{;y)VWCp`phfUu44Yj(NVJE?JiDu3V0Jw zQwU331EIhYRMKk7ST)kBy~W}z(6sbUYS%q|Mmbm@0V3sDQn3q}Q1IXmE2C(ZW@#8Tlfg?ieObK#s%BJ5}7xhs0UR%~zJ zRY<_`p4}$;HXTkg=tQ~mAy3CGt=77O0#o80c_r4q5!V(YR=tBHHm+M1yeQYS}@8=uOROcrA-+Zqn9n!9tNiZx^4t?`rfq+V!d?jy8v!(SDfUW!mK;+ErJ1`^>{~$t z&9a4>ncie=eY>@fXRSKkuvMMdb`%_NnF;7ID9mBb<{RlQ+|GRWxm7lfO4PhU#dfNV zWxSbKw2pDy>4fv}rMSYc%QpaQ#>sD;H~uAtvFh)V&nNJ>-xS-bN?1VX4N$3VPu8$v zT?UDGen;_D-}tI3IBS*r7T$%VN@@hpJ44+>F%u;jFcPFl(`BDl9)hdDcylzREIpyX zvjmAXLYuqdx#uHUHmxx=I(7!;Wvxba$YkKvv%6rr4al<~saHsS!%JFjm(*gR{~9Mi-}C~G-o-}kL59w(YBouK7^MO3eoX8Bi zCVpKTsTVYdO)%@u&TXQ!zYB;7Shhe?X;q86ql#d`Gp8iusFMRD4SGJ^15}JJk6fcw z7F)k-uYKud{PuLXd3>jusIc?u&e)*GDhRQdJ2sk@^)j?3?}Qei(7leiW}-d=;TEMbbgRj6m(Ik|abArvN%mYihKy|q?PVq~~^ zz>8U}qSh(P9QD$Yulj%ep05v^m&&5Uzb3Kx0E!q zOuLDdR%xiUg8aPI=n(oT{upXyLP(&uY-CjwU9Km5(am`xhZfiid6jJ3;d*XQH@{{2 z>2I2Gqd{@ez{iGVY8BvhyhN5GSx`^2un-eOqbsCarzSh%s;SAH96&m-ZafBgN`w@E z`GrQgmx%&b&Rc~5%(Y@hg)NJKLD+I?OV#Hi3p8fkfqnKtEk`1EYCbxfe|mZ05vetzp~m5k!|| zYiW=&&&fB-z|E=7FfOS_(0-~-K>oW@%$hwYt26jvy#S~V)(KJW;}%+uW1ac{*RnY{ zwjHVzgj($vGGfJ>yMF&%Q5|95%ak!f#6tnGuNVypQO%#B;E6)irY^{}>RafW89_iO zd24MneRrj=LqPR&I6AW?+K`NIC&c)Dmi^=)On4>Z(1|D|L+4tx3I^Z+Rh&x_M#`N( zCA{X%Y|_Jm>@lC2-R`dmQIZfr4qB!FSXO#O)ufxA@O39vx%TC4J>o3B7#sX_cs^tb zp5vb-I4E+8&c+xbVb;}HG+e*IZiMf*i2pE61;>q%*(GxXVD!vs86Cd2&&CI$d?1m1Y=>8fSCaQd{mo+^ZFJ*3JRj0^(AY$yBW9` zRR)Y?@^RhGzJ;R+VN>=s%gLw^nt}lk+p@z`xl?(MKX0;5Jkpzfvr~QPWclh)7LM$9 z;M)cp8BwCIW%jR$>InXkJR{gdAt+k3LQPF}9H=X-7>OuWKc0u5#tM5VPr8HTe6*@s$Z$m1Mh7 zOc$`m^hS0w6M*V0UuxKR#_fM1r(Dr^_tqo5%DAp<7Y@8s3OOlcb~?qZoNNzx@caMG zEdC)o|I5tck07-4;?NSfOf3tn%YRA7?$fwPe>>TruKHR{wBPx>*quUokmke%(E)Q8 zg^`tNRv7@zSUK5iV^Y^zW7_pE;RHhSPTIrvbyVP`_T157ijk)3W4*gUB!oS>{Do}L z{O)rXHHJM)jFy0HU?nD16Z%aZAW*&=o(5 zwfsG_gyJEQQQuZT2PafwJ9Q3_wQDo;h=96$QZmb0M|y^xv3@@LZ8S<`#GkPSSi7Qt z%4@EONA7T8zCTs@txRMe%{7bq`YsJ;RcRI(eDB2b9s@jTq*G1LM@3mw0#<7AGpB9G zMesj5^vA@5BFsD`iAve$2JtExnKTEEoC*ws=oY7r>sU`Y`px3|m$r2i&E+UEKYpp5 zjR}8BTPF1D1nzJnc--1g9!W(nU`o6(`!T^_?cNrRC znTf4$!4xf1B#2DO8xw?Hq{o3HEOst@GCA9klU-TP04j3q2EKp38sMdAi3wiow{344 zdHpiT`}50)0FnjE;I{U;XffT-03iI60Z7!%G#;mA&(>1qc!!~N=t#4u+xZ~zg}arj zZ#c3q==DMszzyC~Hu=xwR0w#Uq>jQ7BFQ9ZyY=AmboOf=WN2jm zB3g0)`_Rf|BA?HoNub+zVMNboKK_9uZVs+2$pO(Lwky`#_gh8uI8Jb*7nvZ9&BQ=a zcXGhB45OLO%$PZw60?a?H_KZ20BuPc!NS?9$knvzS}MMqBU=i(`>Eu)u^d)J%d)lM z?YumBZt7?%GP&f7!I9XOV0wa;eIVIgU|`rFNyAsoJHf%4qMC!0i%3PX(ezgzjA zA^=bfs##6F+v z)kmaf_?hJt3vb}(2R{yyQjfJ@#p3oDS2T00aN&Eg!n_FvLX}XtmBO6ivfR0NlUr7u zkJN$^Dppo9H*4azuFCkUuMN<`%1jcs^#oS)ID6lg4+?bmhbofAAjM-PNGeYd7!}7U zVxJoir`&R~;Z&Y#OWYe>6OHM`mLrVbAZ?fn(PQezT?HU*&uCc`SU|8jz)?QC&Qa&g zXy5aI4E83jy;YW;@#SGMUs}r|L5>Psm*}~aeF3W)*SIP+qiziS$rNG|v`x>EwVXB? zRm4>~`_Z$?ILbwFp1O2%5#*qC=2e+!-Ym4l=GoA_?Wj-~EVsfI6dDB<*ArH>ge=$U zT1HZoDm1vB?Bw|I+!5hHdD)#HWm_NWnnvqwnzyx5yNuo?qU-zb)F^_KwL1{x__TMB1d& zFTA--FCC>VeN+12gG442{8q#{&Ex$V=bMNAykLFN^Wrv9ysp=9b+)(SJTZT1T@(8Yk zQ^@y|ILuaC3QUb~o{Gwt6t`^sPgqo!=mvniS41*#fff5s?Ln<*D%=Q%6{x8+-%D7G3YL9xjNGp{bqqTZ>dB;tB)o^xvEOGOE0z%n>m;zSKF zMTCZSzEQQ%>rXC2DP0NnA&xc?ed(IwY5SmqTiDqZHTeiQ1iMvkSs=FUa%tUwNW*S? z6)G++-V|DHk~k?ZZ2{uPE{1dck@oIw69#uU6qRF+3L@DUht!bf!m~QGf?Mo^?%P?| zmbs92$k$}~3GxqUP&jip!V-5YX5_asEA@v*>V>0`CDM5ltMrCv7aoaUZj z-tttA5yl#B@e`h`!;tlDRu41yB;)zpA#joAS1XachWDlnzsO<(=|wNcyV|>BAH=$; z31zKztnldwb(kui}!rga4ojFBlO!^fS&jixMt>)F5V~Q0EjITQ9!v-xUlJwj(*f*AnqLnZim>RN~SRG=?1{4bhFgjrvd3lYqynFHZctZqY09Dtw?HrWj^W z`#zAGSS(Db#_urOA?#4Ej;4C?Vy?%I$4;|9R`TyE!;&>k5U$i3+Q<|#h7^6K)Udm zZdgy6ElPm}S1B0OlgrS>T($D<-9=gl==bJPF}-)6mhAP=wsU7WjALVSU;eTG=*8#% z@y~PG{gydA(|v`W32)La*r6t#5P)|cp$OSV-V2ZZ;~|IGNY%o}3I3vvEVFLi;juA` z?)lm73NP(ar_MO|W{JysFt*dg0j*5Mo*;8NMA!y;x9zE%;Z&g?T~kcF^XhD#>y%Gl z!9e~*NHn{x+GbH#c1n6@1dabR#hEk41i+k<^R)48Yn;ty6#s&}#um{uO_xYEHCU0- z;gImbWFE#1h~&E5M0}Y>(8`lVxHpoeb~A=A)pR5u#V54y!%y+`tQOkM$Pm)V69}hg z3k)D$?*2`KqR7YsCFSTT>!3ct-aMm1v*78^=mVa;@VJGs{mzQr@|{_a-$%@*M5rAW zCC?Er!q3d1B7D!I+Z{I+$J}S#rXCuySZXWty|Ok>Qy&s=o~sYzXttmI%|w!B_zeC2 zOAXf(Mtn{?dU{5gv?Yd3=b$cCD^NvVePk+1pE(fDZopa>%~qkBuHLlzA(N>k!v@t*I*DTD@M66TO*;avJ03T&MJ$z0Il*P#wGks=MugHHCY9*SfV_4VUZ;-heDsSqrm)a=iOTaV@)4WnU6N#ZEv;JP?kh<% zxLKzEEZC&=_?~k@3{st1n$UicudK<+-FXM1k&kYFUN|okZ2~$;6(!;NU3{iHrU^ zltWhFdq!tswhjA5;mII>LKm*=^J)S%gg-l^;}Aur&=mDjccFJJ7(*&+`dSqaxF zwA5s?Na#%QqvU>FYy<5f7&XNirEqlBtB|GH`D&9vUKT#S9ecTlhd$S{Z>p^94yt@j zzbxzHqXvhC;of|PD^ z-~RF0_$1*L)u}z+ee@0rwCsp(W7?}>)%ATZTB%lFU8mba9M5<=ftIO4NwRx3e6s&w z0?+J;CpUgSLEvy-0WBZE?gj1Hu=3zh8G|0mFVFt7zWbLpd~y2FQJI8+P=>R=*N&ww zdLBFDU-Oas&!OM0BIub^$B8N*g$SPzmav1zhUGKWM^oP{d?4FvX79v)!j{5E$D6c< z;bYTG@QtsJRc}8~R#skES_%<`zhGFu*}ta{9ToGH0)Q&#B;EG6&MrQvZ1oC%ZkjZi zr6qJsnsJPFm|?HQa4_F@?OR9VLqvltmI0^;%f+g`5?8tLmwqR@q(Q45H8xtmA@451 zPZi*Mv+c-U!@c;O8O(fqqC|_#UYpJMEZCJ4Mhq|Z+Cs_?pFdufn_e>CYn0v;KsJq^ zj9=PIkp3TblH>Zj`Dg3a^)7PJT^xVj9lj%%c&qs8qV}B2ISz{L^MVIY{d#;_9;>p- z5D55Havh{DTJ{&Um1)oxKii7o)aKpx$ao{#_(nX1Sw37lm|5xaMDzN+`q8wfjj3|M zyoRRGBS3En9b!5R_+@5dC$_R;V}z~c9naU!^%pC;ZI2YX&+sh!?VkKI0sA*6kgeWL z3A=Wf$M%Qkhod?(P?uJwQRFN+lx#}c)S}}jXno@(-60TS$;0M2QKp3vw%2juhwpy* z(F0(9;8auUr0-oOMtVum%kW> z6Qek_`3E5a<(9X~7RRnOWMf4Q>+=_o(|O7{H_zF!-E~^txWE4Z5tpAI#!=47{xC6= zcm!Yn)~d34aB%QTvhoBAZeaUInnv)fEr?Sw9epR(c`lweWT*N`6e$LBPFp& z9I;hOF6?SRW2C4~l#qpC3yZ?*`!)WX+D`ZxRyQ{_mU)}?B+FROU>12nl|xRD8f7~6 z6s7oli-ODfHYp6N8sMTOKYjYNv#$H2Y<0zl%*vMvVc1}}CckZ*-dZPKkSfJT7=iBNFs1dG&U2K7D<*_LP>Xg9xk@GG$b`@rhaf)87?{M4(Ck{3YZBLpih1HKWL_Gbn z_6`1(b|q>C@_qfw>i6$&ZfB<$COXQJX`N>1gSG_G@I0^{uN^ZFVK*>pbsmHbihbch z`;4%^I)O4Cl*_Vp5m~K)2?=$k`eaJCCDxf&EDTo*;jBgCKmT$?ero6Vq*|gp4@cMT z++W|QFBrF%B#=o5F-o%9(Wr*4>$M<{E7DYOK$|z}PfKpKFr1#<_5s9S$}e8*H~fk+ zMc2S8402X*gE};V>FnJw?TC)#X@o=W!?s*kIIS<>?D(`yh~m`P2(ZB9Nox_P2XewB zw=Z5Bgqd@jxfe`7dUGP8S-!g&WqxI2q<%h82Irf+-!2DNG3IQ+VrIlhIHh^EZ>X}X zZJNkx9}@_DS|3_HXGw7O8wn+D+7LA(1r35XExL#<1*(p;z60g1S|WBPOjsxT*Uz8- zQD^=_14Yq``#N-B(J5<@7dmj#S>9(j$#9a5XgD;SGrIuxk-}Td#XM~5somPf#lt7= z^U%Dkp!4+HwJ*i4eAk{UTUGsV=a}Kzw_N>u^f;@$#5&wWRh>biY8r1azsW=A&CVCa z)??G;qPT2`w5eSpL}r@8CFO2;*&*fxxRJAEUEztwj*`Ns8;y0=3&W1Z&lX-J%2Vcj zXcu&;&FM&ch@lWhOoTs(S9m>I_Yxf;paMzLMJ1z<7}Y!mk+Q)#!uopel(|l6zzlWw z%JnL@)(GF;5fs6`!Qe?hI5&5N_AUNz0gXt*1{sX`nDU@}LSd9#p0nkf z;t0W=TLavE$`OBjo@`%n<Z=s3TBqmKNK7-aD%=bb@yz=Abt9hOY8i>R5Gsi##B(`*zy1QLh(rWQfVwp2Okdrugt5gr2iJoev;DDHG z(Wm2|Hq&Vn(+C@xw1t+wYECfmP>_1p*K)*tDk?$b+T7BimjOQ#QdS%Vlv& z%5OnCyUcoR@@A=CUR4IZ>j0Uq5CU4buPs%>qP6}WHeWygGe*GTQR?BhBG@@gKAlX6 z1}pE3Svc3DE{K&AwDt;0l7?AUp+XC3nTKbUqob>zzG)uxl>>3(>U{y#P7@2@bu+B0 z{R|vE`_&3J*N@jX>tK>xFFdVbl6mp_ z;F+TB3bAe4`AV(bbF*!-PlvEr87Q9E!C_d1;Ye6p*AaLW%Ki3_rPsEz{W2o?^G1X2 z8~$|QUQ!9W~LUxvw4x%vq6|Y6tG2?T5Y`qAg;*`}+RI z3d3aswEIE*mhl#kXOZ`Nc6!Ira(@pP+^C0tX-`EVA)f667^8=L;HZ!T2MhS`2r zJ($%_leTti1EXw2?1(-a*t`Dmnf)qbk3r68eI({^@d&F6jgq|5FZR9Iwf*n6&kv>; zj{2Y9bs#t#J_O0qWgp~>fu>)Xzq>jsob>oJrJ~)a05zdErETkFbMQsC{iTAb+2kUo z%lA1{mjX&y-zuUd`z zq;bHuD)sX@s!8sO&5|z=OVco;^w6}X;!n;SaD^*QXw@)F{)X6^;0WIfA^9e4b!^EG zZ3(MQtRI(O-+HBcflQNhWb3(#%PYl7u#-Igm8!A4LJ4rz)=%xZChj;9BtI_b9!c$z z+IoSETG@PM)0O~dQ^8GhM{wzdzDe-vVgD0V#{2ebmwa~g)mB|%NsquIMiHZ9!tiXz z6X+4#gdXkI8M0u3Ke*J8553LkcFWjNvDp2&4-*wA7u|{C-x*9$Z4F`0Gx=iAd~WYN zQh>9V?aI5(V6|-R+|Q|^mEN>MgCcy2ef{Gw8H0_v9utvs=+qu4slUXy9(}pCp4dgkLL-5wZJn_`{O| zxe(T@wP*oIjyCGt^kQWV-L&Tsr9>f*jKP%|Bro1Wp6(Z`T5O-k)4GSX!h?K4%THgl z%Ic4>?(X$2VfbN7d0Tp*kI6*d6Dp~=7U#9=lK5q2NFQ6dfSrsR6+83K2ko}ZK4<~! zx1`#UVJ{2gPARZy^Fx-T_U4ez^?mEz2D3N6|ngmY?7nPbW=C^e94(1~1wlxOcRFJjvs1y2-juBvXi9KtR5 zabF1Q*2^%&BaBvayEO}Eje`u(fWXF5l4+XBWH_;HrE*{x7yF=GXBkn}U6!>pP#XyN z=Aq;Ibs?hhs6vIQMSHqFXu(y1CefC&=(kSp{P!tbi7zc8B(frwXTJ6Dom~5Vm6t(dYcS~|$cTADvV29-2 zIm!k#h@f}%k`O;q$95YX$x|^$mm}Efwk$k7Iug=8(20^~Ow-PdLL4bbC(5`I$~P43 z=vM0Pq&-S-^DbZ3*;u7vKCp2L?v~Q5)MF)oF;FYZ#Q%3E0dVU*IuaHfs`27LQEmBR zl7ZTo*MJ#gpUdo;GGiaBd~znqvZ59+dTSy2q>ZYT5j~EJl>?|T<2P*GEU`$r^lr`x zvF2Eg96$-EW#fI##DgWriQcG#_7@r3Cn>Jb|x7z>(%*^ z^SwONOSA%67BIy)v80YvU36eWcek2Stmf4QcWu)glhn78di&NV7NwnMV|hZvU@yMd zPN7DpKW7Brc%bjYDCt4=5ablFsiY_e%WrGHKH#;Afm`06wv4gu*7L8QB+r5hxM4pHfnmKcy`=pli>i3rvT?P`~~9as^CIJ#uucdG6ETA_&`gxO<) zp;%Ti3j(fH)Ec+#+$26{wdwLrS~}&d2%f_%=|F?b<@B99%YdIANoDCor&=0PKb8>- zq)oLR;Ltu}@X}-gcJ!#+7|Tv|)K!TH{KKxp0en26raY6a>(079EomzZ42^Q5Mv~Kx zE1?M3#MSb_N)M~SBsanH;%uv#dcm|sJ4Rw)8+`kQJM>>BXrS_4U22YnK)1jmU6uLl z&^?1ZwfQ0}P&uPsW$AK{mN2!K;ItimuY5tV)uofeRQWmckT9Em+DEL%kJqi-+dsP1 zQpqHBy6$`h--o;Gxw&*i5G;KP7oS#8fPZGSPMfL0Ufz<+tN5pk$v?4Dgu_85=@iw)q(lcBayk$C5 zoov5!fo4h;$E?Q+$CNMKjG2R~Sl&~wO6K!cvzc$Cvh==IZ4V=(Lm;Y*E^sAIWYGCU#~md?Q1689?rY!L9qtHjJ1hb|w^D7kn1DO}M&JqKbH3 zV$ik1@D&&0u;xkk;l3BWgUK^?pKaIk`BP~l*hych!-Nx&(nP^wEX0kGl13g}tY2~B z3RK}F3fe>fMVTlSzfsmQY>Z^awPdA@m3U4bdv-9Tgg<$E&3uh%vx@hxW7$OiW#oHp z9k5pQ1=3}E78b1gFXNvwnX)U-p|e}6lL|OV2$Ie)v~o;*m@HZPTp^vEP(g1fT^Gif z6!Y110E{`nZTB+!{aC>#jc}DJ!;bJN`3b$n*@PjBM6s(}I?d~yYz&9s!*YP$!?P9!zr~-HBZ+kSntt?O(vrt77 z19?s!7&CNj)v#qaHFO`GhFSMR3$<&W?N-=4^Sp4zKK9Bsmxd^9aA=9;j=+GJQfW95 z^3IzTHv7@T3p;9 zkk@6p2)eBC3M~s#NJ)RO@L8Uwd&Qec61eNAd^EI_tDsraBRvcs>W>UqG-|hq+-+uZ z2Kv~`yE2t0FqSLhgH)mS?%nH5sA!tbZIdIZ?HM)4mU*Mzr(&jCHS%@S0XAmy1U2qi zamV6u>PV|!l^FXgjBxIL#DHG?lNFib$IM>*=T_g$us?E^Dfb}s=W2u-MGj1^UfHkq znok+g3rgcKGXgw-vPByJUd&vDRjlOMuEuk4F-vYJt;lx9s|gZtOd=KN$R_d~v;^&L zP3Lpy7Ez#WazyURwwYAq7C|>G0I!;wi0jHYPwII3MHwDMo|9~<&Q{)~2xs+tExE^b zAmv1W?)(s+^=hT!3f6`2^p8J&r7Q&xV>CV(-DkBKHBwe3tXJEv>cNwW!*HLnW4fCE z{v7TkaDJE@b0Ak1t28CkP-i*7M8J38z_+?1z*o&_C=>~Jmk}S+2>#8neYA4h1T3jz zISg{o?s7Mo?VY4nNnCK#BpkPWwlCWZ5kg&MbkKz({EaIgl#86xb3X2ir>HXO+cYfl zHpv0^mNIZ^KUr1e|iLBj;si;j!_P{I^Bkvsl;S84oXRh7~H@LkTW(FX(cF?rH;l{)z?#>zfC z8*`euV!`!R@X1=w;H{}TWs|;m6VWl(ve?NAZhS^Ww_g{)p?DsX)|Bw(h>=a!R+$LV z9}mwYIT0CI9g8%!7S5-+ye#aR9C$Tp8K(Dw`pww(&g_DGhik{$#C&gs{z@|JEA0iY3P*`~zm;fqN zdB->2FSP~|?UWI3C;JWSNvVJFt+qvcM^wi(9A zzs6=rRDaGY{5k4*+3?O=_?^{Mtz_0fwNQiIsE{uy1r`vDNcda+`P@JG<}ZDkFN;iH zy8Z-oe=&0`t< z&nT+{xG>LWf;Rs%CzbsINHv+C#XkO@8B_jP068?=7dTJErjfl^zP}$G`U~4>scI}U!O%t;9&fqVgBRvWT1jd48U(3!=MiPt0KAl5W75Ebb`kwtja1%@Y4F20| zbVQBgz(eNK`f?7Fo%qF-WPESK9e4fJMuKZ?gVn}&?#_E$tI2Vd?Y#nQK;ORG0_WbH zY5L&J#Nwd$jt^M$)DM6RXsfL+%fCmp{4uvRSdDRQcSGb?Ca+;1!VdMkU(8|r+YJSD zUw5M+b06AA_58W;t56aX|H1?i^2B0vh!&3R^*o6dW`ee#iOp?Mjf9y%EH;rXSD6tx@qoXY{$7Cnszqcakmnp z3W|xWtss(0DBmzTTx~LH--76emFz=zrD|Al!2{#q{#n6KH$FdrWp)9smJ~I%3gYS% zFJ90gQoH)?*Gg-JqKJC63O^SYm+pDKs?I&kJEBL574{&AZb9rr;u$`f`A4B--0M)& z0PQ9(*=c7BDj;f#yK!vKvY*1e7h;hy1eRJO-ZeKPL*KI4-&P5E)gP$#Z|hz-8YP72 z*jMJNU8p{ zexTpwW8kf>`U+=K_QeHk*X6EUcyW36s--xD1*4c?cnEIj&Nq@W%! zTR4e)KXDCN1?r9i<`;v}yB~D^&2xOhi-b-x$E??%^lZ0k1LwOP5MPmgclkDHZoise zv^CoV{RkqVdQK}b?c_2sl?7PYe$S3DKi@!Em+1;|_o*`H?90kkbplS8@&eAjHzK{< zcKjLQ0~jmo`z^z@#^TBZ8lnarC!c_%KC#Yc1&&KW^sRY@L{8gJu6hHX0+&pRb$SP=a+DG|0EJzR~hJ!qM^sAI;UH7{E9>? z6ZSTV-OCTha|735TN&kf>Sn2Na0nC~n;OIZ1b;j%Y^CpBDAflPJ@x!cU9WSHSdIx0 zFs)R2zdO9l%j}jMqvz;#xV_w?q1OqpC@JN4JOGT`bE@9>!}_V~LbF#qeC#2S8JF#J z_Z&1=j4ximwAPAC^_x{VfcIUjJIw_COSE~d?_->9+v6~0Q+@2`9!?oe2>vc`lw7hT zi0^K?#`B^ z=5S8hWKE2{VK0kMQ2(IRfu~5uWjh;X^a5c zb1*eYhtMrWxVc)q`Gt#HhL|Vyk6+)62_=cn~{6g~iMUMT< zzrnFUGQ?R_u?@L#ZSkv_(ZY23j+qFDkVb`@an9;Wk6NsUD{5!!fVh9{s+&k2A&6u| z)&wkv#*R%bWtK~|Tcqx~I->#Q*Z=;qu*!GRyEyz2J_UC`?_+-tex+nCpEMib*?12X zD|nq(u_R-k#q&6e^E%s_Ld{f-!S_z{j4BS4L`8M@%3b!AvJT2Vk9r<*)D^CEr+9tp zj>(O;80k8hBW-w}=djd%98EN_dw=dLEbkRD@1|R^YjTfi$a`Vz-)aHCd7KgX1#e_O zJxOJ^P@5?9ywE+YRH|LhJ8^Sqn#-T>RB7LdV44;Bgp`#4`g*y$#oicpLc&`W8ATH= zxf_0O8sWS%&6FZfN<$$oQ=AfZtWL6h?x^?f&8R2a&YwNPpL`gE>aBvb(S^fA-$M}r z8i1crDWW(*u7g;*7SVA|8r*Q~2upr8(HmWMr!Tsq_U_$#l2+mHr*AH+?Y7TOeu>NxAlc^Y|IQ*ZKM&g4M%kn>=KS!h~8nQv#Npa7Egj9!+AD^CBz zE9oWYO+U9}7!NRbfGU8xxtF=hW9N=sAvS{2{ekBOZutt~_YO{L`IYoEk3Mx#HBEa(g0=-60CRGC5AzsS4iX=?FlK4CLEA%EjA8D-1 zHYGkUHpCbz(W!BhEH&=iA$D_0nc-Y_Ohz_bgZY+;c@t|LnpjinHHw6Z)@4}&%CT+7 zw*Dw4KIa{zX^5xu)+Du_H`|1(SQAEpy>T$95ByqE_=rC|`%fP8&mZM|-xcB}O$RnKYx|AyvZxtO<#h`_w3ie4S7VhMqp5Xf+!}2%9PUf)p#9S(s*9N_LiUJi zyCoB|tN<~-hrBsA`df-uDodf?ktj*x@lbZIil1on8xt--$X8VaHOGdVBKis2*do_v;R z2}glJ3st>1XjYI)9xIp4MNgr0?u4#KYR;L_VML1wSkbw%O}Z)e#&%jKhM9!Wtwg0R z840JPVmjF^esk>t^A+ufGGu#%_ta$taVWRNHzK{n)cdW&}Fw*W|G3W$xcFSngV3q)~U0n>=VSQj!u{ zJVD&_UZy!03b3eC!Vbga(Tu&d%A|7D3iZ8h=1<}YlXvaXJK>BmW_Of{{zASs?d)r~_*?2m4S6&4NA!uT!!>!;l}Ce>H^j~ z@KN9F21RRV1V3`#^(aKS#}^e}-0alx+t?Mz<~dhCk1 zoo5J;mRe7HApr@#)mf^=K0XM}PiS^e-`@ekn(r`}Wu6Iz*@=v{lnH{m|2)|>!`YaA z_J1TrT@=1JXO?i{j2-d}FXN&~PNm5|DNW4@lgM88d=7lObCkM0JNs=@C|^MZ-rzt( zt3sQ*HoGAT^`E#7CGuL-9f6aVLDKnBb52~(55%fO6dTq?AyFUG9_)W6Qr-}xep2

#ooL z8G@3RYsPf+!?)k;)czm`CY&hQ7OEO4u@s8i{%k2!!HgzKk_ZzBbMUyyAInSK@G-j% zwg1`MPLpV!{iI9=LuPD2<*DeKZzMXR7*4F5*i=MQiQCl_t}x43F#~9?Lr;P9FiAK6 zoqr4?eIy=mi^y4%>T+aTQe{`dxkTi)PZcX)Fz^Ma-(%hv*oM{hrs3O*ppwX4$vARmZBxky*PDQlJlK%s?=Vz;9BWL;0s3 zEtK4KQzwLJ8!9ufb5H_2hq`U9dMUYvb?a^?M}HiRjW(&)EF#l@3(UXr ztnj#QQCho-ATb1}9>W=S4qae|6k`5zz16;yr*QPGlB;N1HL1;U8(Q<>tf;=k>qvsK zY(JiMfMv+A(yihssO8M-3>Rj)a(HpD#z1MyWknw^qi}vYuLY3lB<%s35w!9}!>a?$ zF~aYQi*(+*GvqDe4<8PRk9Ch_x>abhzJ0s<+*6d@YvGG+%2WHR3l8&nNX)}^K5d(s zy{N?$zpZN@Epq;&5Nz?=x9?adM$;+vbjr8$K~s1^+G<4_R6<1;fRxO$vMhp#&seVj z5apCdZF2PZ*1~$|^t!l$8zWbQ=B^7xIf52x0FM3{e(TvfcnEs3kb0-zeTHCZFV#?nmYC9Vyf9KJvs)@H$aLD3_h>^C zF^|0j;sS;NQD}_3;bz}^xVhmG_Jbury&!8qPwdPsC;8@G)9miNX)y|wEq8%Zwj9N* zpanCKsWZ7_4r)3Ypt=D;W-b=8LP)M5(R4bx09TEP5a3e+FbMoSrztFjo0>}&2>P^k zl3b0J15kz`(nLIO-=^GXkcR;jh|kG}%NyI2O8i6{prmpgdd@}8U%$S}2@f3gykJ`E z2M4M^KTij|VbAhn3qk#d&F-!8leBI*z@AhA_Bhz@%SUh_{T9U|&)fd!yK&5>a^u(R>L)w%jMh-c=;kj^%%O`qril+$^0-e)_z3YfqjPlJ%i{o1 zcekyjSB(A#aj3<}Lu!do><&@4qf3P_QlV!BT2AaS5ge3GRv0x7G@V1fFGo9kuH21B zR?%2@VFavpGY{7UY7d=Ss3Qo|)q*W>mlvLT8Vxk+ln&JE;VU!3w!uRxX?PK3Sr01H zQSN|bx}s>RmWB``*~gJ;jL@!imX_t*c~$JRzk^XqN4~jz{&F~`w6cAyX**h?T8rKx zxnwFa6sOBFAsF)ub}o}vWoPyFbNz!Srr!G7>#ygm24Axpb<#4ZdEM~W$W_?9uc7u( zp=MVE&`O3y4B;xzt%H?(=H)#g$4}!d2+_%Fw3;jZ`mtGcx!LE6^)!zllE*>Ke6BpE zK|PP#bReDyp9x&mG0%KGPVmu-?V8Ws`}NnHdkwe@_0%OY5{+gX6DI^=U(Hmv@-i7Z z_BwoQ7CNEjiaJbJYD1qJfWdP2JLBLB1DWRxLDp!EcUGz8p&YgK>XSdK=RhJne6} zK%i)xt>$O(cyn?FEH`0CmV(SLeLEC>MQhHwVh4*ENSUN4yZ;fLfo$t#9yVF!hFb(c zqKs0cMjEyLmceq;VV5EYa^Ifj=tw^bI^jUFpupH$7Kmv=(h%+ml}?mnuBs|G>tXW-AMSD^z`(~IMmdHSNO z?=GkG;9G0$hQu$sV!LHY?^3B_=*F~WS*YF_ghZ`%0>u%HHjgBo2xsq@Sm=7L-8-I( zCE_ZQUVP6?n8#2Vjzb6dmBC+cr6&(|j{>yKX-gPFo8cn;tHn#L;a16&vJz#=FjH+F zyYB&xW)ov7CwM<#ovPlvmF@CcAYm29TQL`HBC8ot6yNx|WiA4sHfw?yByTj=s82`(oy8z#AW7|8K%K74nUS$E=kVIcoWO)-Lagn8hk7ewDkZ zOsVQ4gz4m1fa;=Z|2rykeDQ@CkK0l4az9x80fAn}Rl0ac-0Kql<;i_f6U{jckr2Mt zJIT$rzOG{nuSNX5#ox0+<^Cq8`WD=MGN070mR&c7;E>R1wDfs4$+u7D|m)#$9n$?_@4L9CIuSY07gITT1>LZu*diD4&gY zFttAX7>Hdnhi7p8RZv7bWZ?;`A@RY!gzFqqP_1k{C(FyR&6#-VUYy#l-WtB)8h1YE zt2}xbdo>xq&Qo@lIn`tqS4k*X58E30hu%`ZQN& zyUo1`k6Gi*xYyM;IidWq#*Go>k(0&!Lggt0)Y9n>&u?DAa1D@pwm$m22P<-q;3`$awG7}{RGWBxrWZ=EUl+T~xNMbr9MWlG05BVib z+X2!USzKd9^XclEheYg}giXp^qlel98`G%(>xJc(iB}B>UI_@p$3D6dt*q&(eQwl} z6pB(eS55z4a2aof)!Q8UEG5wUE0iJ(G%^keV2cX(s9QIMJ_Cd;SxRN^yZS084FUGX z2#8;RpFC{uRS2=sc(@Kl5e~Dp=y2*Caj81FueWkeQY`^%kaO>QnDsnDCu_R0xB-t@ zmw&3*uA#xPLna(wA1`2;hlb#UKgaehIFo6BF!;U9MTg^0W45BMrZ{AG>dw*KJoOM@Gs zA_nfDx!r8RFJta$29_hG4LW%aH~gF09;IO;Q?v~0Fi$i9Y05TwgtYv-AG#_yfK z25Na|@)%?pfe+5Dls#X+DAa9urcrj$ndCaM9bIcak`Yr2nqyx=sX@>6>yRblh%;xS z4o;cpV2jzY;UiA5;%VjkP&G^TJSW@JS$=@~K)-`D&9Kh?IMS#mVV37;OR&azaDyMS zy?+ALG#J5Mx!N}^(dgkz?C&J2u7jvZt7RG7IPrJg)k*O|h>8%e%*BWwS-c z1t)-50ic)A32HceTo;+JZSaKZ0Y2Oke+t09Ph8yKKx3}R87kR`Zc{IsV^+8v@u1;t zu&q0->omnC6WlIKPXS~O@o*_#bi*0b=)385OYK#tNDQSN4cEg@!p`qAfFYz;(8~OD^$=YOhKemyI^Q_qA5qK!JLl&og%Aj7>WiPqSjYIlQ6A6%U4o zuFr$cHt~dOjTc-cT|CzFoEl?P3 z-i=cHkeRNrrkp){rQhc4WIaw;G{IsS!Wp^Sq(?rM+i3It8`LRxIy4bcs+#Gx{Y4{q zlqNBJYj#cdOM}bCgi@V}02|y~Jf6=x_8|*yqPYz6#%oiyXDO)o{m^6Xr38yHiVk$C zdEF1jCrQjL7MuLeKOjk7-$J=e*yIgaWgZ@bRU{QgJP==D@F7o*Wa8LiVyDS^f6>-b zJ!V&xMoRP2bf7 z%>e#YMaz6OT{Vt+Q!yVk+JF^;;axzXIZ72`kr(fL)&vhR_U7oTuebHu7a; zs~;HG>CsP)yHndw$G22Iclv>+$1z>7oO160tLA0!yDr)C$iW|6rAlP$gHe>i>`eWu zb|*I#FgfUJy>Y)K;VW}oGv5PMtLF*-3tQDU{}e*XH_F*%o3#UgoaXLZXP7*u?r@t`yaW%Tc$RchoJAamQBz}DxNduYNK zp+)_RZuuDl5j7c)eY1)(!s7iDj<9uzyIlF2b^_HNES&9dujoTDIY)v-#B0fF>lmtZNy}Bb7{XU`>(^qsO zG~Ih&(~Vn;7vPQ!Lsauj&vy@M%TfTjQT#^Ip){Z&TG`I}I`X*AYoZYDHZsfS9NHts z=DB;1&e%h`QR&rDU9hB9J`UZ8?`8J3&rs&E6tjBa3^Uksp;W*FBEgbqfdS8G!eP3M zRH0QDvluU+1gI4(D>Esd@2Vvx)O?hmtdwrlc$(*RCF!MC#XDnZ$1m_uO#`AV)bcV@rNe|Faes4hRg{STW3xv@oY2qInx18v&9evzF#AJU>#J;l<;|pB z{V10R%**9kisRI+HxUfLv8TQhO=>CBGLMCNP3r`i(>y^kah%|@WE5RnwzSGH%dV(w zR-ddUG$WEV(vuB^W4EjL$Jk5j_kG+@tQXrJaBh=ybKVWG=i9Gt*SvfH8#tviui9P`@Q*zVW z{t_ap6vA?^Jug%9@$#4%ln0-H8ZSL*RC?l)^NK@~(MajQ6SrA2%kLUZeNRa;9KOOU zn5*ny{!BmYY4R1SLN_v5Av&PPZHYG|fV0D)lc?gY6wedNjBptGSin40*&!2X`&#RJ zh9s5b_m}!Q7XvBMQJaXgn(Onp&@}wkUIB^!8=}>u_XvaO!R?ba52a<4{r7@h4m{y( zbz88mrMOe5rT?xcFJU3id4r*P*7n2X%>h^PdJ^T=snBx=8pmd-9v1X_Su(bea+zE_bUeXpD6WXpiS}ZMto)wmtf5-|nWE6t1uCmRD|%Up8tM z>NVl*RB)4$I_a*3nJB93oK@MkS=i4~p6FlU|1zD52;SE~#H9mRYqk(y1a|W?d$}+* z7amIT>p+d=f^sS#BctyZ2i$G*Uxng+u)_c%*h4me09N;*iRCBG7z?yVu?v#^uqa`q zB2pP@Zk%OIGsUn_M0KGk@IcsKd1nYgeJ`36*(D zkXoHHuFfMK3mM95OBT3_*WWVG_Kr~g1KY!Ml)nkebwV^T?t#PTEYlg5JuVtqUj2sm z%SQ>o;m-n2MOc1 zI=s1WO=0k}zvUw?m}9+FarDD=ti6|XAqU)}aW~1D2~=jFbrlCq@;@#f12>@Z%~2{v z>aD*;?B!F1U+QN!hFLCb&QQ-IXxG74iSYj^L;6{3@?A zK?QOdo1J88v*$p-!J-vD^m{J`!T`Eu*`dy1)^DT#dy_XJv6}@_n7Fay7O(FM<~{3_ zGln7a9VdI|)hfeH{@6WT+<*cyDapG*5r5J_-pr_cjPS0MA$Tl7vjXyTW3CJ;nm+|$ z#+fQshVtH)|0Xx~BP9b2r_X2FPgKfkDD~L1fQbnSY;n)(c=`N7#`~(C|F1K)XLdU* zNUi}2YLF8^2WIOKnEI*-ZNRJ@egyv~RrF-<*#no%(^#N}L0oh5F9aM;pQ+X_^qIR7 zvxRBHHrN_JRlwA`ZX{)Hpt;Bpg0jvq5-p?Ee2D$3_Id^6|E=VNXu0|M&V8kFGrK_8 z1|0x%$CWIJ2Q^(6JhJ9SW(3lS(;wOXx<;DWXcmxpfn+2uWqt+kv4_ix?2}CW9K-*z zlE9{|hx3?g6Lo*TOJ8q$vV@{v{nEWd4Jg#(Kx+Am=H9soP>7U363~$y?Xry`fyIq) zsFj6QV*3XNv0-2>J}95lD|9kjr`?3-pb8vjHBGluVp^b_1g`vm%kaRF9c&ylRSJs2 zVb+y;^^k;tahly}X7lo4QFw-AR9CcQ8I@vMYvuM=(fdiK)oksHq8* zVqydv?n?tKHZk~hj=MLH{qm>F;|VgQR1YeEZdLOT;Io_)cgQV@h{-f3zI57|UgW0o zxrJ~7cxzv&qw5#TrNF$Qj_{lY9M{y(Xc=Qy!$NTxMuDvMun&pU+LtQb(M-%F)6V#4 z6rf;0^mgSjss~W)qY?q$!6>71KR>H|E7HbjW+*Xt^?=hz_SzG=3njZLquWbBJ+y8H zlS=?^&Fs71M|#XM(CdkmKG3V)7yOrdthtPQ!KM?!M5wumr`rjcm;pDcK`oDS?&bcP zZ#3ijZF7C|6Ib(_VViu9D6#xwz8^*L8sN^-P|m;QYGTtw6belD0Az}lDvqY^;;-q} z73TM_Gui`}o(i~m(TazY#f~%bC3JTZV)0e_-w4yzMaVaonh_1u(aMMaL<3v|+9@slkm6}%S?Vi_x)nV+Bs8(i?Gv)n;89TuS8w1<1Q8Tx`K^GRv zTPZcM$PI`zvDq693O(s>QL6y;?;`A=`rnpc^dr7a50%`MV>Rmhq^S1)m#K;3lh^5U z2X4Hh&PEA5KzX>{oWb1|C|+Q6lC)vJbE)B4V}^3B-^LpM+&rW~tx$&Qh71K!C=ANi zyedN(@c-PH{rS!Ly9}8KO+a#CLLcd$7z7RG)V3YnBf-uviL-`s_T<-$Sdm$4I(^EU zC`_P-uE~D7%wFz}(Wolnm%?fakJ!jfP|dKjV1X~mo{!W*f) zmF(L|=e0EKSG0Y%C|`F|D5FhgA0OX4N%S_Q@Xc!=s@b+eRN+ZO%~8hvLMrhw)rg4b zuyOjKz!DCKYCMBFX^1-c7k12V4_Raism8W}!mBInQqT%_zD>>?gN>e3w_E&fN60`a z3)P{>%tx5}3c+ax#p0P0xccnbK{B|hqh?_qGpdR~hV4*1G_X$kjC7i4rew7FRc+{T z?fdudRjm70ULQ*vS?C6Se73HPJQ0&a!>DyaOq>65pjV^*Eo_o8LrF1@{9S)i<1c{V zroDGc8ALdnPR~)aRiiPYhTPzOzb&X9@_Keua5!_|9fLYE%)vFPbF|dB8wTss;(Vb? zp3iGxu9W$e@|r+mb2Y5YR6b2TJY2=dmPK01$j~OUDsQvA765sI+tCu!J)!-`N*lpq zTO_B(zYual!X|WkveVocz5vB;xV;zIH%AdEVbnlKP$_i`%hl-`YHJfKD0|Y4U_dQm`uL|%?sw6Jh!Hl0DGM1RY3 z^C-}z&H$G&OHUoMyl$ons2qO?Vef4#e@>5CUtuxY;p~GNVofzk;X8x`U$1_P_2kxH zUOsEG%~fWmU_#D9I{oY5PCEAr{62He$U|3$r7MYdNH?!1saVe$g1QAM)>@fDcw1VF zENvN0t{U~N9|B@7%jRt1r!w)JPuo9({9kfNktj>n(ZA0Um%_khJkQWI-pdT0o}o~c zf|)D!r^5qS*B8FUbA%D|qWi{tp|x($r&CN*!|&J0Qd-Ya^~4ye^2V^HP>cyp6bOur z>6FGq%XG@gxXW~|X3lP|D?V`yP{TMZREi3aS~lEv-uXlf)$HupAU9y=!j~CK8*_tK zrpE(EKlEMug_B0+$Oh)fMtr*iNFjy}`9NEKK`Zl>9Am>1GgTEqAp!vyaA)xcRC4#c z;AL`4mXD7%;iLeIGxP-+#t7ncd;Wv@NYgOHafS9+eIBAFEk2zr=-&M~>+IvwYJpbn zjupCeNzc!v)u2Q)ovIpYx`FriR+gi!5ixmO#TuPNIbM;f3%-hq z&iW&NTk9HG1y7t;*rQ9|4J7|Fz(2q1;`qE6yJFZ*?aiC2aukAVAy7EhLS{t}&a8UjwU3>M=(+$Tf8kr937<;GB zQW^}5B4bC6cAn(7shyoCuJkd;VfLG-d0bk8`F`8Uymjr&9pNv1xE|!-)Nc<#(#giJ zPJ;0`+U(}NW>v}!C8-8Mt?P@}gIS_ck59{XqNY-ONVLDH>VV_<+ep!RW+3cF>0^s->*0FR%ML&Bhv3vW@A?Visiu;v z-Z|ktL&DcfSFcxZ` zZ?$bte3B=#5`?*0jH7-*`SjOUv-i_CFCC|`cnj{z)reb@>FEe5_iqO#v9_DZr8d9% zb>~BgdY940LBv7tko@WpHmp3cslB&7L%UO&7hTmNNcKTFJKS(Rz(=t?pZR3~>Rt+D zr08UQb_y%dRWZZZxqmzKZmboxpnEx1kv1hm}qh@xp*_OR)5kK~zn{5bz9gvTE;W+#~ zVX7cei#3{F104|IsN5cpDz73(k|XB3%cVlndFu{*(=q@1X=BjI(Lp!xJhf+`D-3ur zEl&=w1gXT$Kn%gnFgU$$VTKJF21ZM}EDS{XWi%s+hv63v)!eDUK|Ytgg=DOW$W-%m zlh>us6!f5nPt)QXR?R}O?wX%9c^p5;RVfTJ>Wb1=YDOxznbHk*^M(lEE4@$80HdKd zzT2k%US<^^R1H+6#^V;?S4sugMrStr=EKN#4 zLo&0@bXRVs-W9tqvEu7f&~^+D1HhKaO#vj=43F@f^aOU@bJQxRD=!-e7!#_>&h(lV z)dBSsI`9O54)t4xspo66#s{kBc z9sKnE$qePOLG~}jVLKiI$}jp_HA<92z@h6neEZCe>f0DE7j_6PbUi43H%~s-FNMW0 zr6^NDu))eGsm#sy8*F#->xRBnp+z$IDC_zJtG#Z2`6!B30kh@h;}~R-!0KpD$;@Gn z=_v=(g^7j+EOd3+7v-e4sr%I4Ya&sfg%4BTRo$G6rO({nV56~ zIa7NqAAAj%dq@S77+CXoOqbr0W!HbjM{{eY{M>moUlCuR)-#7^rrPz6GTDR@qefcv zNr~lv9KcG$IK=B6PTNd#?RhJY(k$(LCKudABTw3v)d!1tKCP6w0o3lAzl%yC)=Aa9 zx_uB+A^Z#$7hR5sYVU+(iqZP0PN$1Ui1#!vVN06BJKE=Cs$c6h?aX_>V_EXk?9C^G z*a@m+vXOFFJrpG3SAqsA&>Lh2OrL1(wV_tK9hZhV^+D3C)_SiA8!nvMtC?@_wFLxz za;zUVDgtszV{4Y<1uwzVXeDoVFsfbXBu=}atBl?Y6Y~>aau-q_p|G6YTo zc^H@=C-(<|OTA$0a_OttWd_;3T+r{>%w?9|iYG=UVf?P~OH|{)0ReE(Ht^Y1*9ecx zMBXsA&fT75V?0TdTY@?fmpYOj$Q@@mpIPZnF8vMs(Bo z2NRD@es=`b#b^sfO`p5g{J>Dn+ukHPw)gJ`_*jEkS2xasurkPALNE#2n%l*IBF>mN zI2lM%s!xvXqG%V6ij3art&#Fl9X!AtJ3r3va#Q(ufI~{Bs`;e9n3_hNOX`~WnCW*Vg||uL%>hH)`b(7mz1jVK{GVM<+-eBSaPn&W zja#ficA0xB$sdNP)%12G`ot79om+Zv8tX&|hUu=cwP{ZiqoT@c05UB|A)Sj&(5O)0 zIyLnveIz?&t-JEXx1kVP`u@;PIW{g{Z-n;4!#a8qwdBPwPp=e;RX)t&XrC5835Pz7mnKJLJNkZy<6VgAz{{KMm?++%&T$2hGq(Z{aE zJQ6!e$Dbr?Y#9J%^?#qO@aB)(7r2josQx3@)23`pWy=dkK z>C8d=yVAcJ)BpY(=FiPvHW9IAvVqe$5h_YUn%N~WRJ;sMK?P4;l(ujPt^WID%1_Tx zJm6PR)9EM<_bb(9^+!4JPa^+n(}mwt12(;zIti1vt*|n@2=7IKrIVCU+kYAKUz`0j zh@$XVc3u7xjOO>P=(ht29HfV-Bgvg~kF3TvqrKckvd^#p8)Y)GL3rpw`P*5)de}Z6 zh2MQM6LFE!rubtre|0ySrNx@_JLwYfhm;ZoA^_P&RuQhdWE8{u&D<*=2za99ji8VjsS@Aka<0+uvY1`2VNczsjWPLWRyVt$x`vvud zU3@>!Ps0XG<)=5uz=b{!4??B-Q_pW<1%$q{9qQ-hu+mokhQNusD@eQZk)f9uZg&H1=UyiovoONYt^4&p={4YmomWuB1p7!UF$j07x4dj%zl z_=Nsu+JAF^bxDEd^pZ`xz<>75nx*BN1J=u_95qpsZl?jWSeQb$cJfk=z>xjVOAqLR zPh83`zC&@>1P=axzVt^$H?D^9;`sY)%pr5>b(|8FR)zF74san$iuhlzA};u*A(*Q@ z*Iyc9Mh0x!vv%Iv!(*Ad#}`Ssd~K6c0dghS*Wca85W+}1$*H?-Tz_fu7SL;)CzTN;&-=zlDtQ+0F)_W7eq$P=Y#F_j$t8WEUhja^@{Nm zi?YeqG^ty?w9b}QYu z)=^u$a8YAp_!CvDs-f7OU1Li5`0H9AtoDBqTfD4g0cisJMDS zj==izoX|T3P5n1#{rAxt@O?*7qKx-Q%Wo1wjr>NZ<&30G)SVulZ z(7XoTD* zI^bSpFJBa!%;wTg+(MK+@?b&s4-PN?pRV^^*;gWm#OeTh&Ly&L~ z8>mLmLPj)UcocxbJI$Vlr4&UCYI`Stu`n8sy*`~;6%>+>fse1w@9|i_`W*$CcsTL5 z6B20gI-KY8!5DE5F^~CJT7j}~`o^G-0TgTLDI2mC^!5TSc+B zLFj12^5E!b6oz@N!Ror$z$EZo|GU}DgPP2jqwMt4N3?zqiJr8?QV->mTc8tgDgz{+aL07r+yoB%`Z6 zcplU4MD+<=DVhWNfM8mqK!E>(UAO7Y%?`MCXJPx#*g!Ms4_II*0jq+Jd?Pj3 zJ#BO=x(0Bg3@nk8ark`i5&g6~b5j?S+uGGHViMqPOdfQtKgE6V`ueP&hQz6#l9#12 zRyy-tUkn3AI|HVIX7gy^g>i3`N)=Ykrx5` zaLExs%y_`(o77QnQX6Vj*sKUWZUz1A_1+ECx67USy-5P9 z*+T-kN{8Lai5%9xgLTdd`JIVDW3t;6Iu{SC?Tz*0(oK%tX!l;F(%!D5&?9#DLZS^36dJF`{%NQH zThsbvfwpvn*SA0>e|srUOaYbv9+odrrv2FzH;>S~QNxMBZ8Z*Sh=GH)L;0LV_TYXM zEHeKeX>T1?)%Lx8D+VAc-6bvEC@n~LcSv`4iy+&NN`X}VBPNp1#t?uN#~*FPAXj(jR~)7Q-bjoe+H z51$Pld-npH)pElepSHPx%FAU;@P5?&STK?$y`?Q;ELBc@6_mSS)dAj zT^0G}32^55w!_F3pb|oGWn;*TkNkVM%TRJ&498Ab*0ix;{WehHupWA+AS_rMgm6B>q{JepvjK$zDLi|wHi z6P>pi`xaX^Ce!jzus|c3m&4g!DnQ(dG}hS9Pb%rV&K%B#MyQM?Qz-B?il&yFeBZwz z2xF7Dd#jJwE$dq7!xm1c7u29y*Qid>zX7UA9muXKGxhtLuwD3>d~sype1Z(jBa_^JB`w#i}%zfh}DX;^pfw3Kw{D z2rXKFROe$zGLm1`@9C0qw!0dE;A9W|F>hxz-=HIuxln7vuKoyT`->AW$U^rbdpzs> zPPGq0+K2ca=s zG_i8}^cmTK1QiwegVEUZV#ikXnX{2|J(3ssL~Q3-%)7Y=HR|YCSRVedqg%r@0B{eC zE(Mcf?U^J2j00FKRp8YoLp;4jn(|4YiZ#3*62wAK^hj(feN7Ev`9IO0vBx)pJJms= z;&f+5{`2!t;{~{E-4WSd-I0`fu9Aa80{E*ozQ`OB^D=p2h^VxhO_58yNtHe+;bpbo z>Cb54nA`}8k$*U8WU_U>eMDBDOT}G&s({vZR1WuMd^@`5OYE}A&(%Ze-3_u$Mo`Ch zrc|pvv92H|wo!+P^R=fI0lGImz`@07)ywYDui>Wa8viRCulW8@@ICHdk6SSPkSr4X zN+aDhR=QLCBfXYij-wxr&gd>oJ?#u{Bra^^CMDAVbnjoqlSXnzhnv<73Rj0{Vqm#tyn(oZ z!SxKYBc-7i4S6^GwB)iCDM(3=G6KdNimBa{DTCgHSZ>hr-c)ca3K15PRoNZcm(6BH z5=$q~t_n{POM^m4z#PGWM3HP^2c@R{$rzNIbl+;8%7m2eA^Bq=Oi$Lk-NBa_{)pt) zgf$l@95Hm+W`N@e^QH5W$C4G7vh-rrqG~{_OWzN;@fXU0Vh11+=cf&kt5Ly`j4B{* z=B=1$Z38W~7sW^Sr>Ek+jAHuAQXPZd{%A(fcdQcqjjtfHYbzxMz()D?l}&=eE}`$DlInb+l5 zbW877CypowGO@5ajb^7Vo%<5H?P+zoJ^|r%pGdMY21J^4qPHN&7qcixmT<2D*GC?y zK)G3xU7H>W#S$eH~KO;MDfCelL9CRN4TM&S6NwyvX^VpggkKn`8WOn6bi zerHIxJNrT!XK+eP%8$ZHsn{l&jgYIDwu{G>ZBO=w)Xj+}tKoSJ|w(SSY4&>}@{*doRy6E?(F6ph?r=?#S z$jZt-KVN;Umm{C7e@$@MsWoS1I>_i+2$ zc9THwnu+UoTX?`6VWr&}$L$+)1$~|SY-)A%=FV^~C(ttxtL1&T0i#a9^viZ7saZqf z0MeAKvKfa812sCGQ>6$t*2dosrLH=hjWA3=mcv}q`;smOcqe10?ji>S>RAIl8AJ4| z2YkQZ&F_o18O}a%Z0$@I>3p!HWUgD(K)0Wg_LAoq#cNo?KExs1?YruXNu+o#*|K12TZGy5yoxA3<5 zO+dR4&-s(Fqot!{Y(<}w1QshL$~@cRO4IX>hjk%rsUoEg7hT+M(F9}Yh?h?UYGh4M zE~Psh^?;c&+P-d*J($L^$%Q5ya}e$~V7AYM1g41um89DYR7voHkJMa!j86)GSn_FM znK+sea4Oom|{68)Zq`e`TW(-1l44l)@o@#h_+&t7Iybo3Al zA>kdCB$-~i_o(yIXVo;Mw`F)mBGQbs0!O1`E{EVa8ePm@JuY(rm*cQ^6f#NzKIqR5%)m5%FyCTy7QO!(^=^5=gR^zIt}ey;0OSl?dD*J_`( zadDNu(H=tu|LYnBhxzdRc9_2+uMH?6J4t&XZ=ATTkH^v{{v3HO_Y@r0^z6A4CFpm! z`3LK45e4fPM)6OaTKk1FEYd!Fzi%nC9ab;LYlB8?7>~%@TM`5z4oFh+=rxfreh(FY zyrKCgFiBLGU`a<2Klu-SP}Dv8`z#3X_I`zJp`LyZhre@PV2T6U$LX2S)Wa$1{WEq& zz;Q&sQw4M)dy8#}^rk?Sx9vHq+4=gHOzZb1dt)$YaG&M{Ijo&B1j5mP>QJqqn_7+W zG||X~>_Xx;VHH#G&jZNU4{RO&d;ngej~#ujN3CdkN4v`ith+!zAo@RHlkW=-@Edg{ zn(u)r0}Dj12e+vwgKV3$Fi#GVeTsV~EI@4;y$wsfiv$B@W`dgi2~ZAayDigpv_{X= zCQ4S`Z6WQ5XNj4`FM+f^Vm}( zX4Vt>)~ZgZU0_Ij!hT>P`saRj@cAzphkHsbu5aC?01UF$Q5Y6t>t`>1#aQ2;hb`xP z?tyCa)Rpv%+2t{RcjG-M%Xq@O_8PlM z(e2N6^rI95?m=PSUqPWHt`d1wH9@mZS9`sKO%YHO9IyyH)Zn>DEmcMDR$6LF019&s zs<#ba@B!U+7+9^Zv#48#(Hckdcoo;;5x3!>K<`-}_d zepUX8e7aD%!Q)UMsqSAn;cA-N4Plk_&a=t-DIw9V@!4--q>(tX&E5=mqROM5s?;4x zT;Rw|?3#xDm(kz<*w0HuJ{1u8R#y;M3b+6w3s!CAQb5>Z{a0bjpTBW#@(VgNEBkMB z=#OglT0J>U=tlRQ=zWIP#ZW150FQ88cPLMQPdH{6@e$jGo1~6Nl0oDw2~7zIX$OnV zujt%rc`3_BU>c->j^xA+E(wa!h{huVYIj)a`@@l=Az=l;;(;&CFWis}9z0-^8Yr14 z(UJ|$A}|$CciXv1PV}kr6I0J6(qtVh(L8K&P`3Zj)G|GD92JA=Gb%NGHXMEx9{m0j7z#SoGyn?Q9r#L9XBA*_O( zoq9WaP|E4l)56u>%u!Pmpud95)(hUufSDkA2a)Lpsm0QPJ4bPlqZUDx>hLF1-F0~CSt>Xjkd^OCfyRE{ znOf2JQ@)-b=`zFrAzk(@WAnK&7Bd1@Yt=EM!eVVvZTmdK zEYITKQDvnGJ8Z3=Gi+a;WcsRP%2X^H*8b*{-Y>%m!Ib_G5}Mv#o9A5;(9jnS_kI%+nkbD_Oq8s>QzNj~ z?pyq<*lvsy?xTSC&~o+o$y@VfD($|6oS}6r>1;c#eqbTvT%TcnEwe1#x|wkrOyTrY zsoGvXu{)M4BZ6E~Mb%`s1o9A0ZA4r!d6&`XkSvWa_!&@Dgr|4&L~P!lQ=!b-%4|R~ z6yMC%H`iAnamxokgju#cZIi{mJFy=;mG_c#;fT$6y3W;x*_yk}FM1zHLtv`yp7elo@LA~xB! z7!C+mdU*^+O7K3p_2)XtzDt<48(6u?f^CMACrha-LafaYR=`a^WVgEHYUcjL0zBEE zK!0DtKcZjTXox68My)+u1%Xl`^RB3g<8pP%fmGbAf=O(z#u?g{7ylGQ;46S>bBjMj zjAT50@PnL633vUUJ#D1Ley8lj$bmuT%Bgky?lnPyJU>1t3m_`ng8QjeukC zqEYyutouMwktvz-X?*}iM6s5y3kOgwaldt3EeA`N8!&>8%#V1nifP|ZQ?k|WAsz9zQvBz2G6jREuc=-#O{1NLlO5(l}55sk& zP$_xs(f)~z`)h7H@Azpy5~BQtq@P*pdt8X$C52(Hc~S`gMPcSQMIqWv+du@gM$f>& z1idq(bDB>XS(3zHg)Fh|P0?;ML9MAh1C&k(oz?0DPq(~g4wbo z6EFOuJj~hd9Pf#ZzFK>q%l?*$LD=5eesqr7q}0fHnBN!Uz%%lfGht$vd^by3(qyVn zKqQw^F=-smrgvDoxek0~S!MKF&I3)lZub6CJ_~j={S%ejBu@L)N5RgTeID%2Bd`hg z9IlQ1&4=ZpVwiy;R(TpkyBIEBGoSNl6>d<$+jX?tS(P^4JFUJUU1w$cO@(+UP&|68vUT{O_4t}b*uK>t zkQfK{<2@tBPlB>P-Zk`Ysk0?UG0vIQzO+QO?3zdkG_XlvJK~C>y&gYO^P6I3D&^sa zC!?o-lyR|Cm>+8@k?s>uZNx?*o#z3V6JFUEn~=%TDP|7$rVVy6=uNZmU`oLTPL=O^ z$y;P>T}J=2#Lx$@Tf7^!J{qOxyOc{^IqCB(0zdN?jBWqUghvakh!8gvrb{v~1f~%V zsJXb$V~pTj_2Zw)4iX85#w0l_*Wj7Vj!}zo#|V%t?;#g_)xdRTvxUiU??1krycAAR zoT_j}R=m4~56$r74Hp#9K+y zV|ke1%YZ+i^Xrq}p)+5+px&8uS!SfHOx=mL9+_&f+}Ss^;pacqhT|T&UQNF;XQOY- zIRI{Amt%LTOq`ZX9x1DE=K`Wspf%S-`vedi78LN4-%_g9e*(Ho;K?6qMj=o=%vY=! zqvyDjaG+S0MCf%=%$h-p1hyLSu>k~XF^LfYse9zSflviBN&L(u1I)t5_PMVW zV2^;75b|T%;g_+%p4HYB-Yr%u!z$>$#y)j7DxOYbEi{6A(zoRQ-V<732lw(CY!_UKtEGyagbV7wtgUk z@i^xWJ3&Y_(NN`EvqP@}*cw2~4p?`Ph^6bL_26L1aCgcL#_mra!}&w+&U{}Pm(8+X zsvSW4e&vD8I@_D^l~9#295p#dOcxA)pC5B_Al)b@y;YWgByzSrqgTZj)K2smZuzkK zr<&BYMd+tKq$%tF(uaK8%WRDwV`bwPY--IgmF=W$YS&~bgaZh?220bNz_g3hR9tg8 z2_;Xdo|D&ACRafcDz&gaMwC~nYaBmZU`!Y2kK?L(b}H~zDf#h$Jwz0pqIWuM)I7UD zEP_Qyu#9@rM#8U^q%Mf2yvHH5&E5Q>1Bb+4%VBO(f^y!uwkHECB<--LRB8BWe_Jg z8W0+;Q!kNoLgo!(vHGw8i?jIZb;&7FueiRG&pHN`)%B8nx_yS)<~+*9^jMZc79g+S z^|V$l%Rk4WDyn2lyy!)Eeq%|O0p26MxGqc}$Nny(C;DW3+ayDyDrbNMwk232INP#6 zkv$jiU&t65jO2M9A@}KBeG)K@AnX?$B}+tRygOyy%3AKAH<~>JcfN6YkwbD&lb16? z^2tJqd57dnJpq3to5iq+Vi~#SVOPormB7*N?sLwM^o>32kTkUIesw(ua0Wjv1fC~S z1MFgZPGO%8mnju09uAd6J=-gp$Jc;KS;JYK>Ci!MS1%L*=m6Q#)K2(Vje70(i<(k z{syi@dt3ZUJCm&|My5!5?Y^)l9ADNjqZ#Q5#p{6{+kVBW(S~Vk6)SwW+LWr*2pSz; zuR*zwH5ADOSMvY`CmaU%1NPpV35EAcmxDR|E$K<~RYFhjc$VD#Lw}-`9vG??k!TIf znG5@jPm}(%>i2-CJiin+UY+Q+_3VditbgsF;x1rYugiOkk6BPEEOE?(%&k>`irqKc zJ=quGq6mQS;Nj}Af_4O<-g6lC(`!!;k1RfkBYKY3%mGb*(f0_5F1Q~i;i-+xs75~s zBG6x>GE@Y7L8ctv3>ZA4J40?gfU3DaQa2t|{E^P1fuVXVs?#bmQEFwGDPKH9R=Qx| za8PL9%bCjh=a%>e7I`JSYu1VhZAi z^lp(5$XuJTyX+=>+fHA(!u}#lQM6$Q&&RRu!qAbN@FD+~oAmu1wIJV1UAjZ(nSR{- zS4d!PIai7)^aT3fy%=2C6T}qvQm_BO13xISYh?7jcI;xq+~oV+feiSV+99%qP*GAm z$p%t*TwX9id;=&q!2a_+aS;HYjs#le2RCb!B)U(>$IAbY>G+?c=>FmrHK2})K}lpf z6Z(cg_7fpl$N^2%q9K}MaVT9iZrYC^{`QD%{kJstuYZMudiX5WdxhU3@%v{hhO>oL z*U9ts(WY`VqO{dI;r#kJNf)o~cV*-sr-<+O$dv12Qx+Z*#&_u`-~MB`Pvg<^I2Mn7 z?%V%x5%l0*y!ZgxTk!tHT_eNq^@=sx850W82+XRZqL0sh%XR+D6@0q@{L!BWblM{l zzUvk+U{IuHRL8my;IpH@>uUCSR&m9(|1rD7q*zMZrK$4U@creljkNedKo@u0G$C(z zqJXvwifoB=o?RuoVv{>#Pqx9#?vH+`G>iKKZZCIIppl5BFaVz=hpsN)adPBh3?TA6 zK&ghjXCGcJk9)YRx;722&N;cneBuxG}&BIJBuOm3-^-2lTV7YuXRT- z-kaMS8zSxgcpp;P0^KQXVZ~ID{7X|StLBC)>qTGc=9Kd+=~N1!^sAQ|-qVR?^w#lC z--MYGhyE}{bDEz}RzJ-)3M@E!{gbixV{nrGR2~8D7vEu26%=?F3 z9mq%4pmJ8bpU8p~P9noym~gWYK_|f}n85P6 zl~NRr4OkE$?@-Ob?+57zpO1jHISz$06Zpe050A-zxO~ur!6|>d`(|0?M>4>y`EMG{ zp99?cQ{w?V?|$#}hV9&*+v_XkQe)|`tp|7V5Fixy_q;EY00)daFISRcp$$31NKTs{ z1VXWxM1eD1646VnNl$i$g7$S-c*`<98Gq9qLKpf0S^;r@7Iie6UZ6}@&h{-(q|VRH^K7YTa?X=PfkCli zn*^@>F-6Yn@mcK44SG9!d(KfjazR4My0LU{LLm%)MV!houNWTgSgh6{wMWNQ2S3Gv znekTAoNH5vIhzar2lO+pDz!U4b$%g?#=wh4&{Lpt1kYeHk+!3x6q1tf^O!=eh{}5V z8XWMQ27*KH$~#(aj=y*^yFUbI3yl9LmNM6>@Za?G(zBmh_7_6OVcI$=mw6> z+4B0#5*|TKX7lyu(icaOr+jM1=}u+SC_rXR9p?i?{Ah4oiOu`vFbjj^Z_;jqV95d? zjSFOREq>aE0bLL62(Qw$**{W;Z&3h?Fj%v`FmJ%|Y=%-oy)~GdYyrGM@u!~f|fR?ZG3b}$|-L*(3>U`mHbeM2MUk& zto5>Iys6~#yO{O;#|{eMn3a{3!%Z*A(MVm(zj=+b{kb8qzI%-e)`T$F5&1)^7LQTo zb)IyW0L8Df9?#H3Y~kt$u5?NJ;P`<~_1PR5?ct0`aV~Ce9M-SCv{yNa3cbZtFeAK1^>N>$c}qJJxb4nMQ^TYJ|d>S1ctsnK|wIHzEJes6z$mp8HMMD6Nh!t&EjdBF#9@6)|}512T#ERNSYTLa{I!JvHxf z$e&oTo^mA` zea@K7K*rF?^utGZsTMOcGcfi zjCd*df!f_JU$XmbqlwEXu`@QSj~Fb-c>P$#0FluVIL}+VkH_f6&+TNSi0vu5uRpPH zc~{T$!gPL~P1=PX8KhbV1BKMf6Bf8cVEnHzvu^rN8}WFSmr=dc<&e>xp?GjO zdn?KtCj9tKbT+G1NobH1*Y(w!iIct&|I@lj)&e5k`)fB+YHv8B1R2y@^SEG zyLhe~=$WJRaW);l8_RmBv+i_OH7JqG&v@n(IpTUBb8|SS&}cMYF+1?;bk7@idA8O@ z6tGU&ZjB139rNW6q7ZTWAYgU;uo<&b0+Q@3rR*5C2A4ZmByPO5>90=Ovp|EJ7gNa-hM~;v zB%m;;lTz)`arDsW&`C8bHudvVtGD-bZyLz?fc?OGK()c1CRI2{+a1}LY*UJ&pU!9>;`yD z^h389EjET*&BecFz7D`()fDnK3tJcGzpAy_GsdjfL+L2Zkxs>$B@v%o_px?Nlh`Ko z{z4EFJbULK`-Vz;+qC;DF!7{khWMzJHfsdv^wo6%-+Z&=>b>Mx9frzXucmJe`^pDdHQ}o$)2?85!-z|gMX5fa&KoEDOibZZ!Qes+)&?a65y|BWy0qF& zcHiRA9mvb~QOfr}jZe6))KHu)%!*08I7Iwd>!ZnPcZtG=`4;R5+0hN#MIi@QGB#Ot z?n{lc{)v7$`OurYa#)~u8wb#(t=PG_@U>)4oL0}J<{pSOOAG|3;&fsKluf~7Tc>|F zMsuS*YdKwy?$d`^tEUDS2clrBN1atXEPy%IEV+!u_bM2kiq!lW#ryR#fY&l;r!uyx zr0{Z!{g7CDq5o5U{>K6QAzYE@JML!!Wl7&`#09F1PTC)>Dg%Us3LcBtJ)uxgLshJ{ z{~Q38FH@d@!~>X#5(BCb-1tet3kLcUm>)Il?u46Gar)Xn4AmwrrUQe&=G6=RL}Fzk z7(RG<937eyu=*44afwbUE{*29C|m>SKK^{GY!O!Us;m+?lf8w%JBj(qmi}AZDiPi% zgG&W8VfGP@9V1mMOBWcL67g+-LUK-xBquhR`@#1HGXBo@%z$z=uNBUP5$mtgcb`F4 zf998L+d8T~(pjqSn>pwelJ9;`CVv@EAZ#>DM7|o#`d&I`NxZpjzVr)TIW(*QG&XD2 z(9UhC6DdhfS>;N1y}vKA+i_JLf7n`b^A&xdHps}TOS3zP&HI?V|4n8N0SRtS9oH)? zCKj5>^V-HIx^oq@04$VDIhe>AQf|b7_KF= zymhQRD7u>HsLl?U!Fhu!beUGbY@UE;^uJ6{*N|BbV>C1BkCj&%LseZ3j=Uh^_`7l| zdW~CqHTuAT9}sLs#eV6J`bPi03%vf+b=`lrE(&#bx6>*ulh!7Zt{Ui%^wxzf2SvNh zllIKpSfkqo*k#F@lFRFTeMwn{GKPp<@~?W&gX~VP1&^~mdCg zYh*Q|_48s0RxZaD@&;W} z#2VYOY)}a#HRqEj?&2DB`IkaU&z=Q{UZ9VK72RK=Lsu>BMqL|HAq1NM#M7%mIz-IKo!Xd62`@5jGl{ zx}@W;hGH{b-gOzAiR;R70UIjFdj zVE~5JIA_F7!82<44g8ZW^!L|z1%maeY|26aya;39dV^_g7f@?dmM!lP*?JT*4|X24 zoYd7lmf3+GR?jMKuDX}79{r1i^{@AkGUQ{oZ(Vi0Lv3yC(osJTp4Fz*ILHFM4)8%x zrxH%l>aYXhFzqXtDu<_o(~SA7YgJXSfJD^(sg2!*(rI@HxJ%t38rTt%(*?uXb8Em7 zpB(aCpbZ${e=4$f^v!V12LD z)^w@G<(sQMW?K&YX zxT)du;K&!Ri!FoY$v&pM2$iLKC887(4Ih*z)Y1LSBQ#TU75_CUElugCFOmRj(cye$ zcZg605BJ81jKAhM8k597x~^n?wRbJH>9oF&RV;J#POL$k{zaZp_94poff!I+88Wjk z&Xy1cK1UMJ{Q+mmFtNxx&QW3QJ~eiUtP~56vLE31SG@SgJ^=n$Wv+JM%WKd3fz*iM zTBbL&AR$&%%^#S*q~$LT4Gr!7*;A1L;r_%S8cmX(9!LZXjQ)tmo^GAsH)B(Bdkg+7 zYGdMLx~z_Yd(IVagy&}o@}EUB-obQs+sSwkM5E%zF^5E2#O398^u+LNuPq}or>-Mu zBE|Dco2nc=KR1KP!Cn;SUwR(ys) z=+kU?pe|FJEY0HKNcX`7Ez;g7k9HqxNafK5qYfYGpBk<|p2VO3V)|}Auvrq*ZxXu( z(;OV9(s<&{JOpr}Nx|PAOdp)7(yK?}aX3T!2)J{^mf|O?;DI#@A)JVU-Ou}?J(!-S z*JZDh%vu{f)P(a^36abr(j?)F3A2tufgN1vJJ>y&4l+WdTGnMPWdE6mG^M_6s}JlZ zS{92!doUKgsrgi>gIXpzw2N_l)ZS@R2ff{XMd@j4_%b9~n9Q2Fvzr2~_K~JGvVZTx z!xHaSHKJ(77I=-II`voH^{v?W4R#?nx4v%tbG_lAQ(_c7}yLX2JdpWB8{ClhqGbLnU~Y2#!9DQq8WM5 z7EjTR)!rjrn|xVIZl+oL{^6fe(X_%D~6BX8hVB?@DNJC`nO7 z5<;Rwldxj%Eh>K*X8-#R{Ps;VsUSl<2kj5HIzRRasu$P=RI4e~{|c%Bv%23i7R4WC z^&KBXV%M|UQ_js{JyV=@FpmHfFZ4GB(T6x~ze!mg|53_{!DcUZMg0E{Rx1*3$z{qo zRn^oi%sDY1v_`3kI}YcFBU>4~X(@trGd%QHzs9CiQc?={#0R+JHM_k4#?YN0iE`w* zA0OKT1a3@roX+At?SKc-z>Ni4&A&kXE$_Lg1EfLWso34-1ZikR8>gQ;xBVq6`j_uo z^mKG%P!5v85gJc?s zUBJ!`wBBw7?ChhG!q zilf^x(i1c*1BH8;H4R(`g9a7gWR^0&fsXE&cL^#G=JN5e{oO!`{w-1j}`a7bzX!3 zMKfdWT{*$D+V<9DfysZoBYUj+yeSzEJWOs3eE}ABeRNB?J5FEPK%vhc9u`WH_~1D8 zA*Isea99clBd~azz z`2G0*sZf2JcY2oW69LqAsP*9@cl9NCR&nO(Y%iRUkYLZc$o?y<4k8prLo~8Sc<=Ed zNKG*Ghd|ZK>B^>A+B?rgHY=wc{kD)AUtnzhEzsRN%rQ3(4c(WDAg4gCSjucFw`F+& z$)!T1E<9-_P0b=r9QSCGo?Y!ILRfdEBo(yRGx5;A>u!^tV^^tqf!!_-u0mB@yr?%E zB@Wu<=7h*u<>Zn6XqKl00O=$Euhh-9;B!fqrl#H8av2Ul_Az3I0*XGP7m0i8bA91> zuV;OUIT?7kLYliiCU{Qrfh>kW-<$J^?>si5e68ig8Bn_GBBGaq2o zal!p=WSjh0Kf)F}A-gn6q1k`x%emH7nB1C|7?NUlxTV$OUJR;VZe!# z2>&N_4qgad zUrnOi{KYxKYj@%y!WNX$-5`i=pzcWyM1KR$w~CVlX{^~=s6Z6QaLGGQ8N#`jSScoVSd-fAeMY*|+x%g9Tv9b88~{0v z5KmF#M|7Gf~zG0eW`Vs3xHf_}zZe$mSW=>e+5Uk=P2P zUk27)K>Mf8fE}HT2Wdrd^rAD z65zG4#2l1vxJJ;JUJtUYxrNWjWvZ^U^pV*Jgb4iT3wP`nF+HQ4=)vlh*R6Mq&a24g zN6pKtGj_gm9t;#(co_bv@(-W>KuX5>?o=T<&_UAM+K5U0cpp-CD4E|+QQ?e#=POQP zfE1WE$8@a=_Q+iJXH4e1FaK}gH*avU>jq!la;u`DXxVeGtjOWwkN_wdiCI_|!k-Cs zYjlIgYO*yKD_uXV|5|m|2!qK{3^I*6)mT|61eg06bhz;wc4wu_+yo-=v~W`C=2<6) zW2>|vC`1B#4)1AL1%zfLgyEhM4X1?n#GE=f7Ajv7glaz2AmK;O4hg4m3H5VT=SCl)JkQPNPcVI&n;$8|^0vj||qX^RGs3#^$uWe{( z6KUlIInT)*BI>##^WA|il6XdFq9@qU*10aK*hnjs{hx!&+>exId8ywl*UVKkb%ap$@(2nwr-d&od%x<&S6Zn zR^D0T2;LUXb!UDN8V#0nZclhx*mBBzN*C3sBc#B2ju~pj*1ea0UU}veN}bxeHb=wn zYrp^|waVhZI+mVUFj74~RV_ZPC3{y(y?sRe*6OnAcF$Y zftDNFj%S*&@x5VRnxsBe-#^23dav@t!0MT5U>LF4x#@BYFgZX1h0OBl?~UaeAU8+# zK6)={XIVDAcoBJdzOPm#@~{p7@s16)$y!1Qv1xrk^D;n(b^FFyr1w_fhTg*_VXSNi z=$lDqA(FuM5SGLL-o#6vJI3uUZKi=L7|VbON?<|;O!4}9?Sa@syi55hBt=(ha~!s5 zoSg*7*5IsE)DRLYl4v@ux-xobhRvuX_E-%H$(lG_o03Uf zb2q(%i=>OG?24v3y+tCXinS&G#4zYcin5tbK*8I6JEJ~a1v@xZmNH^&BkO!qjTu|c zzm!4vZbAlhzNn!m)=;cOo6iT0HBAIG71LJ?->iw(CL82@0yTjYSI`bS;>bLNQZSq9 zueBfKOrx>NHTBtKvMhN9mXbQ((aLa0sLkat?5jPW+K|!idL?1kVIWfM$34R#CsRiF zIC4!Ji;KZ00|(sYRs~OO0k*H!4nMNjp5Vl+wciL!n8oxa2e)_@>LFOF%Bxm}&H`m- z&D}a1;oPEGwUW!D!Z~i$fQ^a1{wd0B2kf4(I7V{;*(17+r~{MeFCHdQqdpB?pP=U= z^+It~&XsGALufNa1u}IMr(!?hHLNcOH$6YSO4pmHyk3IFjJ?a;gZ#3+0G``*u*56$ zIjz=eMAfe=vVN{+eF$UH#O^F1erHWDbZz1;jjQQ4nE(U>|7nbS2+>W%9y2%m7JQY7 zAk$!b%A_I8MERxjW((u0_Z(h^=y^l4W2He9QDWKN)$OCpagJU}>kaZ3y|30lAm>x> zJksl~2NRlxsmxausWX+b`8m z4JYDUXa$5C0ykV+PaRN^wSgZ+LRRVA0lwb<$JUh%k-<)La7TNZr#zTr>^*Z5&3CNj zl2bgA^-coC9<}mW1;ZWh&Qt?$K}lbQ1C z;P8V;)lsISJG~h+$uJpkxTfXZrWN09oOM>JBVJ}uOCUYoMGnJy6DT? z1r(bh8lL*dwM=^Q<)kx2iMq!2IfiPV%Dx>xsyeZ9|nVh3ife^a-r*>i_JA2U?M0^WW6zEg-#g6KU;asv$XKcR;R?~3yl$Z zjwtctSR!WgwJsu{ieAtwTCu&CNA5>i#dy;ZQi)YPK{#&Lw^vW2saKpPKu6eiV=N2F zGxhedo(;)fam6?3*pN;T#Kw;?XOj_Z%8rrYC$`cO^ks{$E24WZaf z3W{4Sv4>J?V;e&4oZ<;m942OtyJF~L^ZVc04_z4DFl|g9FkU@SP_GZ1*)ix*EN!u* z6oeQsWmr}bkWaQ#aYR4c!nZ1_Mbo*ix z8RHD_m9+Z)D_Ql8KN$_}3pl&WT5I*&_#U&lfbKJ$+H&8Z8KEN^XQ$7J8NH6Y$18d< zx4W9P`haOGePD^6m4NmRTsR2mH5x0E!C}7s^`TBp@2W|dD1*s7Mge6B3kEvtXjz5E zD2P{@XY6{dCzkLM_$0Y{1gMv1HoV%vsk6MHS%;<`GM>#sXC`N4E?Op?ow578Q>U9t zb@$NL;g~8``P^#q@c3f3e|$7kZzv<6$Yh2Oy@K2N8ck@#B9q)qGF`-A5nRn-@%i%} zL(daaj6e`HHe&~tSROukp&7#Qr^Z&v zOLCa#9+jil(YVO3sL-lK>++or50F9z%O>M)ckyfc{bOB1Ggbv*Mx^p=A!E~`P~FeY z$kB!G2B8H3T$OIrp0Q|zCErUqe0dz;;uEI*U{%oC(6>aZdy2z9b@I-onhn%*Obow6 zdYy6c=)m_zrN}d`D~@zW>}i zo^o?>=ko*f+VK;IiL8Mo>v<#I%hbx+Nnzw5kq{u~_14)bu&0||`P{C6`fMz!nYenc z9pmLC*2w+ZE42lbXJ6%S4VM_ROqT>r3)L3gHG)p#f%P_cOVp%ZsS=Vsae7x{fL@T~ zjGA?i`X-)?Nt=H|1xd2-MF!i+SGSQK$%S~$l`hfkaaOifHL>`RLnooYJ>%F7HWp#; z0~I7yVAnlCsL0+w?ff#!j!MJ!aHw_bi?5QCf~3XTsOabDFIy37c%W182RtT~t&?$4_N@ z!^3`$NK>5hnKJ-8zxxJBV=)U({hSfyjxVcDU)*4`lJ6v9o(LNp8kvakSkzLTL1a_9 zu;5a9HxeV#XS)Vi^Y_ zdm_r7mdzEK1&SFq{J|-jV8B_iLl266y*&uGA^)z`(%?M0-VnEzow0z_DUeKAJx_L% z+U}^{&p&#%c_>V3Ps3~>92ix_)PnkF)Hk4(V~356S|n#XiMq(o>^y8lln*g+cakGnXrJJ+P6b{v!&eu8o?CG3Tq|q4Di_6(Cp`48Dk`1BFbtq{OARrUqI3vD!+=UNgfMgr`@$#G=Y973+u!%k z-rslZe>m=giTj=_*0t8T&ULPm8b|B$YixBp20W*ThEv8Xoh_NX`WIB=K0JS&@IJrB z?{ZV+BhikJK?9`WpauLD!sM%xNlbWc*}(nO7vU}O4je`)*jzw`enjc%6H=%7;f<{nO?~LyqQ&95sqESF)S(A zzil>nM7L(+6nv9lhq1YaoNS5SRK3pIHkk0^ldm_620d>y?dsTdMD_Tv zqsH65k8`UfM-SJuV;qY`w@>GdZv~&by7$&jpvIf$hA7xO#9tC!tTs!n8+}GMv=?uN z5#7))n>Bg*BvNp`&tmDYP4`UI+B;K9s9!-iWy)4UN85-fr-86#@{Y9)(MDffMkuw| zH5N5m!EGbx6_)okO+oO&!tlilDAh&JC?O3Ck9UgvxVLKX!tw9HbG)Kul%5!Jk0G=t7A1gBg+n0;q!LDc%GMWs%orWce-8$|4QZT7wSllb}n+qBPYe7 zO*p@CpjH9xUU_@`7|-BJIEZ!@PLa@ETrzb_q;Vee6zV~Hso8IGI~dmz0RwRPXxQ@5&1^Wxn0)mUyf;3Y||R`<-P-0B>egQi|6|> zG0WzU!ei7kNBmT&3!h8Pg4X|}uHp-q!1`phk3 zs6EDmm8?`PnO2QWNvt$$O%sY5DWyT~5?s*3Bcjm7VOEKR_pe9F=GaRt*!c{-c3RLS zspFn~Wy+5H2gV83XWE38P5;D z?dW1a3k|_-eUp3Tjjd(Y^FO60L_)tQN3ezgyxfC4bTd`Mi()g|+TsUP9XGKDAB9bcoYx>*#b zR#r}8XxR~ENY1ro5a+uE_i^<>1<7o9X*tz~e|iqbi7C$<9(J77Q6u1Q`{fA`2ca*P z)EUZIkPmHgu#UEUpr=p1B+1}-dh3Km=cSGE-X~L?bTJs{+*NyS`Uxop^VZ(ZBhi9r zXTbt17yM^VF*T&nRL9a~vxHf?LP3k2G>a6E52MIn@u3)DO<^G*c zWi`+*(-JSwyhX@8u7$~N>R?%4fL9Jc{6teH#Jn1xq1am_KFGdvC|(P zGMF|>3i(c2g$S_RreYKj`1f|*KbvR~fIF^MT~nb@@}lK8#B+Ru0Wb@kE>n6?$w z&I_0R=)C!_&EpT>%ust0Y_jq@yFYtq{`)5a+z)-p<;jJMf8FEX-*@!HH+}zlEAQZv z|2+@+lION3U`?;AuLo~FEt39E10`91ErnYt)UeUzHTWQ?I8Z*-7PSxxi081_Jn*HuV# zwrfik^E~1KaS`3`H>$sO5xJ9Liuk_#faqS?u7Td`7nT#ezpq+u8;+?t%`CIzDo^*lg&SuJvC0#ow*xb&8h?NKWSrcJ`D2!?SE`h1jvng#qAPRZhn47U^5Re4Vx zJ{^;4K&FUmbOJwJtojYbmihtX4fl)I>Y0&%3+iQ1$fk$#XxF;RsB)z(cHfAONnBjK z*mLm~@r7Ed+)+KzJ}n;eb20Fe%;tQ3!BnI+=pS(dNHj)qIlQ-WXe0W^?z!?(6k@YOeK1{kimXV*7C}i}W+P;6kim&4D}_xok$Htsg*Akf0x!)((WAd8ZfeLNSNFfYSS? z&zanos-5-*A5e-xp8@^6hdH9(-z5sW-W5043n;PM#*@VW&eMpGpu_MzI`4Nm-n(^N zQ<3woOs0*>tJqEBww|eAWrCnx!j3C-TWT4#yuVQ6I<;SHxcJWf&V&M3>2)NJxgx@1 zVI+F8Asugy<+5dRHX*kj->|OCbkX%{Z&Nn1^&^0CGkPD-4PM+*ETD=97t%Mp6nPXc z^_z{VzW5{`W4-FN%*qnZZ*AfEjgBGZQ2RP!Ly-fk;w$dXEtc~=ph=4HF3 z_YEw8$E+>RkZ6PcmaVdbT$=>pzSH#SC_-U1CmOmK(`Q=?Q>8^W@0_nw1YMP7a8uyX zQ=cW6|x${vq-pu$ym z=QELP-OjYL^{N@|w%e`%^_PX!c6_Ib!|3TqP23W1fT29xSKItSVwGVwaEf6x!{`pa863dW9M1kmJ8glP`a{O(DIdX(ru-sT%93K z=lwU*4OYne=+VBpm8K*oL)u4##!zZ|+%r&~UAEo@fNG^&s9N*0S8t^B<%oQ|*2{5z z2K&f-UINM&xj%~EhPBroMB)YlZ-Ca)0v=O}BtvfbfflVXx~*9K_kaRtYPGDi^Yyj$ z`H|wqiB*Hm4XB|wM*1@8A1A(o6o>-U?*!8vKb#BY6C3n{mf}D>gilDIeJ*iuIDxf-Am*_JL-ToNhz! z&i%vNIvdbY=TMq$)0fAFVeTrCE-ba#tYFs_80Sppx6GM6Ug}3!585}VFD<3vu}Uko zqoa~{S-2ad-3=b@{DvWVDzpWhRAlI(nGC`7V2ZA`gW7{z$^roa@WmNc%y^OO;H&93 z#z>mB^=tZ&GQFV_Q0M7p)`0?}G8}@>yJzEJ8p$x>`^3Ql&%WbeZ;7jf-gtlM@NU$G z*~dRTR>{X8LQ5FzVwgH~B%>nh*$Ow}jQw_X?Vml9Z;47z5As-b?O!|j;LO(6)^J=> z!hLiG$NiKPk6QwtiG;ZjnI<*J_Rp!0nDg&=ial2nwyMAFk-aS0y%beoP|JSp+5lt6 zEuWn=Xic1cTOoFFp0sYDz_1KUGuH|n5}AZkZ2vfUt4VdoQZUG__ zrLnT~c@dzit73rn^MwlJ3|C?-;kXy#6qI_nNSc2JWb~lu<39fU0vbTCcyZcmWARml zN^1-&4}yFkKiDX-+LvEjQ;mxEjk23a*!IvfQ=wls75ZY6hLA7Nt=F zmn8l$Zt@?Fg>WZ4Q{xPJap)c0IJ(H=)@;i-xlqa5Q)+(v@Wi9EZ|rM_^UAlzWgi}2 ztL_!URrsBC+#%oU@C@GDYokJF;kWRG9O`xUrr{%F`DB~poBkbD!Ehgt!l{@EpvmP#b=gy$0 z9rfcE2`lAdXZC9r)v{d|Q*HmCQw>EpEt%OMfFUKzML$QioCKT&T5s}$RH;Z{3o%tJQe zQfx&yGgR(SNK7+MJs`{cJN&>fblS?N2(%mHv2W)adiPaZ!E#<-n3j88ZqcHAzJcE^ zTxPk$h#howhrN{z9yju1YK?;pd#?3nlY#;Wf!&COSX*u7F3IN$8)5WcanHsH4(R|l z4&k$2dv=9Q-C%XH11E1Y4!WUh(q-*MR10}0?xFYTSs#e59UTVLlQLY{nD={^P=q)` zAdQ4$JCU20TaW zdCXf|fIkPg;;@b_&a>v@0W}Zk=5J!uUE1P8H(IbxQ@0cednXq$HT0amzm0lxK)C&@qp<Yh{`>10AM{$a?(>_!xhuOs#HQ_Qk^HUi_0~Qa zeZ;c152I2?(HP(UHaVxk!`8TbD|DxiWzSWF@8pQXV1a8{RIXGva0^GK9w|MdSULW? ztz347i z)j2!Z1&ZY+xiC==^+V%0H5(@<8F&1@KT}Ax^H^)zSR7f{1N_XAidfKSJEguOv8=G}?EG7%B8^I=Zc7${0C1j)5xTIrb8{~ZdurxFE7`U02oY^MydPeu-1PNy zOEhSlOyaX6=}XNp^Rhv9eDVtQ;&Fk3xwwQ;@?B_2)mQj*$$o`-mn>}@%$4h_&r!X zd{!n#NfW-7M8yjyexLf^!h;_MLX`O8)pe!@r$2`C$G`sdk37$@6ZfLqvnM70_2pk4 zp+5RZz=Qw$kiU$`@1y>INAmyZHTfmkEbQ6wUk?8-oWv)l+Bp`7U7{kW15K-Nx@xH9t)^*>seqiGg%+(SF|P|l6)W2C&G+o( z=5_EI3cWbg!xS*1Rr zIgH0}=SU@V(Z)lU>&(~p3IN0LGOzu5*UJW89 z^t-2pUAe6)Q9$6lODTf#(<-)Xvp=2-wARUd^0(*Y05-?0!Q2_gmyJ0DD|B*!L(*10 zo*%P&ODoPu%qWhuDSW8@N&0TCyg+nQVckA^cs!5tRT1e!N5riC%FS_Kv^o%mAbnKF zzO{Tp9pUUb_wAGXSxFyWbvmt|`2!|LFyz;J1*5lQePI{65IY%}hM32_(dXKJaqMp& z6LEvUz?xLtwD3}wpW>E4yQ3%D?gKAe2Sl znU2q2k57e5_)+-z`H{NkTVq7K)8Ip!u9*|1E%-F2^q}@CZsrs!_GIs>(F#bc)bSTV zExdcDos?)i}xwwZfdoj7_x5} zKd>vyzrkOTBlQ4T>ScI7WWJWB&Bepa1M~=xy%$dXrscZwSlolnKGZ}s)C*!hT=77I z`o5U=;P&bIKx8VY%8?FDof}f+b9Gv!LdN(zG3qUpEb6z@OmolE@A}!^IAI0LExR=z&YG z7{BDxHd3F3dYx}v80lT%Ui&alN1=60--t6_av6@XJft0c>Kags^Qo2xoMN>X7d-Si z4XdoGE{n6JjwRf504;QJf`rSuE5F*FBgcph{C6^Th1u3sF3lr|_u6o8+)vyStABU( z?=y&D<+P08%^W%MXo%=57vA>*=U@8N%_B}L*z{%DHUTge;{D9E;B4u0b2JKRcUCRm zBtr$Eb@#6QooJdGF>&i*f0s`4ua+NXF8S7Ec8$e2e*9b{a5KxJTEr`-BsR_0=sTea zC7VTb$9QQ#nxdt#qfP-szP?aR_#Br!d~5?mFsmtZmvzveApON}3~E7R3~ua~exK5S zk}~UqfH(QhWg=hOv^jSz&5amoHy0B=PEtn+TP1fTvhFY9w*lZMMwo7OF~9R~XtqO)iY3v0p_q zLxu%o5rPiQr<`Fe(j=PY0o@lClOx%*Ih6o$amXi}UQC6G-}`BOR^OLXf1pP{7XB-3 z24>f$c)JOOmwzn$q*_{TeN_`a3wyigKkdIi*=Ma7a8^bF0-Ef_!+?Gcizho|l>*e# z@>0JD`H3_ouITpov>7ggh&PNzCBbE7S;@(TjIYuZ;lrKAf`-jih|#I9oQ7NGV{wiP z6@uyRuhe@AH03-poCSG_X0-+4v$kutoB&VmE8xj(9P;Fc0m=`dr*krG9BB~ju^B}(*WUonnwvr zAas#@voFaBqf;w5BmIDpo}ON$Hk&!V?nzR$&;lJ?!U3MhcXXsn{6GgtDKh9E>@?7{ zVVIM()Tfi24$~OjCS|H1k}L!JwC)KZ(VCc=DyuV+F>_&vdbmn^VBMnGmGQ;GfFX{Kx{0D{uH`&B^olin5qv;LfH;&2Y(3r3TILC zr6e?34MD2>XoVlApj8%mj4@=mm{%nq2fEZe*L?ejF5|N((Hc!7y#*dFIZwP6IzeBH zp{@c6(W$Qqjo4~I^WFk4uT6JZPNP}|obqN_!@Ucr_n$yF;MN8b@#_$M*SJ&*JIzIP zx2b_-B@+`1kqbFELMhKU{yEIfL>=a^yGf6(GI@Vp9c?i?m@BQE+tFw17qKZG57BGO zAmtbl!rbKpQed5VXDB!yH*Qx??=9F{tn_T7(sNK+KA%p=mIaR-X2-VAPPdS%^}+J> z{oL#`CSHqU0Jj?;w_&lv+0qo*wt6pLJUf9XGIx`YZ6#4ENEuztx9+CubD5u1@LC9S z5pq89Dh$&4(*T1pex?8Cf>nKWxL|8IvmB~+p&z|<6H!~ldgDkec?nDN9I9f01whp0J}a%^g5 zv9pQmTfkJP1TlM@i3&TlJ%$Bvtyc*fRogzEp~RO_4kNldYx7(s7HUO?9Yz5AhMvmZ zh3v?%=@M2R?+$6uQGLPl$!axTVckP+C*L;p?$6O32q8((V^*PMf#|uF8nzKi?ei&v zO7cqWy8BPuBWYvCe8a!Lr8)JnWkR3=Apr3g?XSqTN?FZw%rYvAGwVud2=NdeJ)vNX z@~~nX-rGJK@1mxdJH0rG;Sv{^@DM;@`#^VFwL=g|F^(Og@4v)j8cEI$`gl2zoX_zT zF8JO?i=C$fL33RBGwbyn?ALidm+$(QPZok?oLi%+;QPMC9tTOT0kig|yZ$>C?B55k{Iuka z$9e^^1G?D`ubr<_9Bv=aeHQ9QT@#nLb33{B-@n~p*jOH?o;JBxC-$J&tc6peaJSo~ zNK^Mkt$3^GCqvMxbCS)Wnh-}DY}t@8K6#C||MP5Q^wf|&CojH_xsKhW}Buk6hoxu2n&V5*n?qPc?DVT2F#gNOvx$IZ1qXb1yOv1lSZfpTC~ zPOC9lrR}~PLAdT)wE1C1%ZnYXt=Moa_M-QykgvL3A()|wZ3>n0I$w(+UAVAMHP$?R z_tuCg=tF<=(5D+qmKYJ2$)1Qb;y?Puyp3YWWHm_)t$fivRc^>sxi7an9wz-t&Mo(aiMpp=Lh}*?&8o5AIYzS|s^}Cjdx5P4* zrMWE1R5u#NuHJCA;Cy%VzN@iC=EE&x$2S{^`;J1dO?7vPJmO!PXN?!S-Lwam4IN^@NqXfq{jdt~6Bc z9b_?WVa{_^Qh+3m?HuuN+qUdTQ~_m-m#&?qr#B`B&U|K9;+YaoKo@_Jv^oIF2UH@4 zu(b^fx1&hlyylS@%Z9kFNCPxmtvhQee}Snkm>!jSmq_I#VXa!ihEmhFo8zuD8nkEo zFtu;n8HjH*$8+ER0YqJTFdh;U6#I+EDWhL+DAumQ+$;F$+|*qk3}!tt0_LSLzV?uC z$Ci7ibCTPB^NpeDTpHG7Zf!ZtZ!+VR=M1kY+4jzM!jV}O>MMPJm65c&-=(@~{tLs6 zf%S|#c-^$l1Ap&XN)~6hDqVtjRWy1}>?CtpnZ;I0))JSnhn1(=wlVNm*2oa7#ggV* z#co!+A$<00&pag83`(EA)vE=GhmRTYb~q{EgPpoc{tMQ1Q58HqyTRodFtd+w3B7_y zm`ri}s}g{T<)1qWU&tZN3=@eSJxyGMeZ8Ieop*xe`5hR&m~VS0APdYGG@kp#4>`?n zG5Od)p)pJaA=sSgftheFUtl8wrohy{k@=SaV*tAJ_s93TW|1PbxPS+8Uo_A$PRhcl ztR!!G`b!eL%kk>z%8x7q3r~uWul!E7`IuY(;nVUb9AQwm{cxp@hH?B7bV)K04*yx_ zu_ZRPUO$^ugI*A4i>WgvU4ckrqeZL(4B;_xA>ur%$bE^p`qzq%kFV8xgq<|HJ`&@MY*y7d7u@C% zcFo}WD0{K9iVyFlr2DT!B#%*4-~zgpEJBdE+J*8pa@5ULuC{5;nFVoX>%*Gg9FC;{ z#3K&SnA`G(G}7#mL{2Rsw3FE+cKd&p+rifva52F#k985FF$9?)q5^00( zYXpx|!9*kXo0$(Dstvg*U6QZ~W zP6g3TmD8du;rMnp;q64ruqy%2u0P47AGvd@J6~Vi3D=*0yHX`dSQS9*?9nWpQL9t8 zT3ZVxqJb8EYh-HBf_G&x*&(f1prhA%`Pon$$*XbFN^#Lbh&~E-L@nS(IKIO} zm2O5iJjxj<+Q~m7vJ0ld36py~o-nnilc|++$RE57FAcCE6t6?)7p!GWAZIjZA z5gu5C|AToZS&#J36uz;p;-<&`J1;aUQ!>#l9U)37Wb^xP{rimk8;X=~26!&%>d+gz ze~#5pKG`2v7g@kNC!R$r|1{43cGX?flJu7r07#8o1MDG{X3L=BU(Bh0y&-9Ha=HIx zXY5SDlv?9I?%bjQ7V-zNWdd=zRm?B>-GFD;F2?Ydm)c?!?%Iz=IFjBQzFPwvg&0mv z=DSB{IuX}$2M7#6<02|Ep)9W2W3?qNA7MVWTQa$|!g#4?rUj|3S3ZB!ye$R># znV+2m^;D25Cv&z|yy33=&f476_eg0Tmyfni8w+z*Q*U0ifm~z09ty^9tKCAurhTV; z6BTlDl`IMh5m9<|Io*7wbFtW@TXd*)f*zq3dnyMst~wF<1&{QnB~WC%jDdJF`6z z-m$t7|K`zZmT`F`pZ)h|=ePxyyc?J!3y)o~!EPYtu@SJ{am7%RQeT_lmHn|C|ygjeu7ZmaS-vL?EfvP6e4u)(s#>syahF zyozJEv>*5ym20ggkcR{koh#4P$WXR^>dJV7!N=cjxZ3Cp+$Wq)sM}RzraDvfkee4w z;eCtoMQEg#;y4?q>`ok!$qaD_Pc#X9o2%`|_53VZ?EB+#gWh}xNB3XIzrI44zT+1K z_(}?e6h=-<4X*dXxocKFe<_?G^WLvg;n^y?gqq4I;|b{3Hlj0wjcXSbmkJ&o3Z~#} zUc!n@eGSS{>KfUWc)U^ecu4Sv^UmcbrGO9)#Vx%Rt2?L{7kXmiS6c3DcR$ZcK_jU! z%_{O+m~8bg#}zOkk!S9YUwKnYId|i+w)%L;@J(S>D{d-Uj-Fo-I*|Y*8r24R>C9V`tp zQ!BG|KBuo`(O~ItwNK@R=t0+VLWcF$cGfRa0=|~r?k3mv08KI~u8-5|u-qYRJImT} zs4tFHqfwG{vZiZ^#cP8B=pXYyKj>#@PJ47S%!4}d$@Mq;4KdRCB^d%z9e@|tO7BPD zBoGM9^?H~LkGBfECCfO~CQ#3<0JtY`S5u}tcXY(JT`C<%=Fc$)wG2)L#-+OkL!;Va zws8)%5zD??!{(4#yC>iIEA3ZZl>2h~qjuJ*5c{V_`fJfp2iWCLN=g`(1ZvE98Z_Ut zv%h3}l_H{(HmL+5zG+9YZ9<^ID7r^8vW@+E@*X)_4i5Tk2b*Pe;QITDEe+p2mg*K7 z7xME>-;};!hzHVS0@}KVv8WpBbcN-I5A1mzSL1JDW(Thg+c+i?ADibh-?_8vWEVf; z0`nq#@Mw~o+07UqP)co*4)m{jCsU-(T<@+#AXw&F*(#hb0E}>!MI-Iwsa0G}Jce|- zblL?*k=+w__1$dr23kgV=4*$c-aYm?`O4ISHa2apTg{JzT((l?5;ir^1ps(cuX6bn zc$4Z%Nt7x!o$*kBaZ0MED>4Ea={pekC-8PaQ_&44lGfwPM!qV+xDY0`vO<31y|-@P z&IOdi9^C~oFX(G1HW$-XmsiAfKJT8u$rEM~tfZC-XE0>TwfkOaZe7}GdBn5ARWeY! zUHNrAls?v+o{Q&wS6olgehF&#n=<^E`9fC9s(XoCdc5`doZZ6vwZ)*tts20549<|< zx=zm?ZyGq503#~#@%Et^TK&y6di6KH#!VuuF|Hh9>;U+9D|rtr{yu;LZXfGD|9Zah z&4>)`)$=8cB=l9TiyVigTOf~3dQCQ;%5_mEO(P>T{9UJO0LsR002K}SnoZUY1T_;@ zZaMrnIGg&$*s4IVo2A-$jE(F2UHTF7H&kD|D~(AI78BFsY1YA>orX4TYUinv06&v1 zPhI5})>t;7%iXs0?>OXtPyUXKA17qHyaLdGaH>{-N14aax(XRr*glN1IONM<=a};>G#I-h*vWR zTxA*nDM#KB{XDhBSoT3y6k>Zq6H6TL2WTY7A3dZcZbzq{3?SHm20J??KYss7mer5*u}IueY&xCHf(t z1X7Z${f(l1STcZF+YVj}yr-Cbd9xW1Ehf8#*Ucv~)(`2dEr8;az^xplgzpcX%9!K1 zBl|k0vsyjDgVZb3B}c351)oEIe@6Gx%K`+a6PIzVS>W5MC;f@p8&j;L#>Nt)F{&(p zLlC1D=Vu$ey|pDDDreIc$MFU`0rps?sfDPnpPz(C7s=nN>2OFG3Vs}GAvoLDdorg( zoXCESPgm#J4Y`cI?oSWNBklA7+z-;wx}+sk4gj?xwuP>Qhlx7yC|f2=P0Gq zO3z7NtvLzQos(zDiza;MgRV6gl{GCnR8=pIpzExLI^pDR!Anl$HBY`?cj`DH%!nZ8 zLUQdmDXY0HFmH}(^sjUiMYt+|t~(f;H<%W7#!sKu2-9C}>K`|6={eb{*`yqdRov8c z^tJe9-2kRp5b%9BUHh{-#n(Gt-t84+OxUAPS`vLqFkV3V{5-uFE}l9jtYuZybUbx^ z)xx_w>ve`@KPJ(gI!l}P^GK1sUc*&$~3XA#A1;OX{IuiMBw6Df7H)kFd^Wo6l%67)W*d)sG zC2_q4TmTPbmJPlF+dFw?cs;tY4yWw)<%=(%!^`Pz_ssxh4rw^h`LK?qx<0{N4Z+0R z(!DoaI?u4Xv4{$2ZeD$2VPj#?Mz2fhC%_?1yN%m>trn*cxTCUAWPtY66xS>INF<_- z_yVb^3RPAbpiue#;CPCq`TW}FqK9l)?V;%+^R|0GJQBj*P!CFlrF%ubFyYAe;;CSn zz!op%6z3mxkR|;+!(esz&2XKA_@LKPW#1Z3*w|_yIgP3rNSQ-X_%Ql%WjD02&=_qS z?J`x&zT>)^dYuvDZWWY6=Uyo7ybeU5nim&8Kwj%`ftsz5gTm?+S4(ezjyFvPTr%Ez*J;F$Ys^6tc!e@Lq0VsR!7BxspratT{YD;_h}1 zQ{L}E=u(0|+A}^A`C$Bt{y^G~`zN$fn(|pEX@{NpdGCL}G5|GiRVKHCew6ROf4|?)a5X9e)IFtn7zy3_?LhfED*j2@d2)&2 z_WaZL?EmS`Oe&xNK>=Uw^zhff>t7y`;mdVNlirhoy8r3U@(Cc`j-K|I%KG`-|9(|3 z3(8>55|#X?J4f(=DpKYSeCvaskslytTsgLlCyyAdT$PEOU&;#u;sQPCFc71O{s&* zHF38ePGRhOC=hIZ+&fsVElELv}l-o+k&qyt&ybZ^(CmM znOtJ|phdc{w_fhrIdF-zq8$*Oi92bdmEk;X5$_pqNK1N_5G=FRw;xe{UUr>974Vvl zC9zMnX~rZE`XFv!P!tPp(r^>0iV0(p@S;K+4UF)}# z$*nIRnNBA4Y%1Qr&p9K|%{no{$TFBR3m7&^ppNHIl}v!X!%wpATGaLDbwZuVX`W?z zJ-Fwv#N8*zZc}9kQNAsl`$^+E`{;ANkY|ue>wM*u^;p=cp{Y&HJoZHu$&GK%&lY=a zs(Ng_neV|v3E}ZR`8pe+qT~66lWoTLdoolfs?b(l%M$O@F3=3pFD}whA9A~6X4qKu z(kcvN1R#1|Vl-l&GAKf&ERSX9^%}3*>xWVvrqSq)+HsmDZf9Da-!{qzkdkG+>%%*5 z8?$54d#_tTD!6;rD)OhbVHhG_7%elL?nJO+(d)#{3xL;e9^a)g4(DhJqm6`&90=r# z?zW!CRF!%aW+JWUdkdBm-P#}CbgMfNZ}OFG->Y?egb+hN;LYL?deuQ6G2I%gzqhwH z)pyj#e3^jl$yegC9{j?+va%8jda4T-M>MZ)xitw4ndmuk?tK@73tRv9#G5>*lzxHT zN#}MsmEg7l6qt3Q1cqfFsKijoDLBt&L=ov2Pd-mqFwp}yK=&;qS)|XP8BX9YBeuCf zD}JA_g?C%o+!Q?8|5+6d*)tHP&;=(tK+gEe_tJ%9abTBk4eUyJzjy zbmN*}wqb5k$+}(b-pMrQpqYh?;n(=UZRg3}Me;&p@9kH4`yR1|8WOYN>z_f++6K$3N{BZUt(^VrXA=PG&S4ZQ2eFA!|B^c#~I-!<>aJ&<=}xf9_N0h zN~iVW(dtW{p;k(R^eReydi=OnbIf%p;b~>@a=j5+7u5Jo&0R3O@tpWWArKXOSalF^ zdZsB-V)<8_cyaS5@GwC@lj(+?xW|Zj^s_8o^6f>d-5u>HF^3kb*!8UdKQjXL`rA>9 zZRm!ymM2fu8;&nX7lQCy0Xba+}?hd zR(F3Cl|f%hmg;a#0nt5I<1q6o(PGg8gPQB9P0vwFy`=@w<>tLUWs6&B*oXRX0sG$G zINbNh2B+EYQkHJ{08ont!c5I@-ZzH()BH`|g~XmjQ|F>+MY@!#T#iQF(xig(`vfe$ zaO+RBMp0vFnZM^X#ki;R7?L-wvPpX%llT>6Sl z8(qUo!Mx^YVv16k+a@~bVjpMlXD!v;-EIX;332P*?~bewvtD|?GYLEY*w%=uePd3$J?jIm}Eduw?CDJIH zvNpo>JqeYYZ`R1C5)mj{1{R8k!&9jx;!WXj7l~#zGjIZn2kw13vx!(81JKtJPXJaO zAPHxBs$FL{Chn<_85ZVY4T(0b?btPbfzHir^-w~dD*cY7nppKeN^-b zCav|JiZT^+nqJT)Z4MT7&z!)=@R`1uciH>Mjj*J`TrYz(SwI&smcni@8xo1f1717b z-?9ZV%oik9)$1r^$_uHKlxjW|g5gG5^Mjzv0JC_nHSSkZP} z&_R~&8Z@~q&TGPV$Iby@%?odluUzkvxwe4$#HrB*I`R(HTcrXe7Gpn$Sl{EGaMR7` z=gWRfq4N^V0}kQlI{@ppyYI5FCD>5P<7#1V5r-hp!jC1h4&Tz$-<@v1WQMj-jlBL` zbfBi@{F8D>pROLt3Rgvm|8={XoOrD67SbY+^e!6!;)f3F$fc60mCILwpgqtD`JNZU zp)nT1QmQu1m0-%YsW%B!pGqf=nKws_X*CChU9}>ZI5JzvHE8|vr5{WDQE%A|b3)h+ zQ8zy%EwdW-6E*tk*Qsjx?(05V4?B5uuBR<+g;zPZ=j>r}0a}D2U&rHJnIa#uYK(Xc zEEyr$IzrH*1_NFRAirSRRKxm7CHEsVU$~VQiEb$$_c0GlNLUW$a-)F22WuC3d7QS~ ztp~DopJ6LGG5znPi9%!4Z!AT!4HU$4n&H|SRd7@2<+>P$Nx# zuAMC%K2Tu6h{(~*zczeKgJm&nwrcMB=IWG^z|EXC(5uhJSa{jK=SwPTf!!!C&AWI{ zA0eZ!4!Ex%LtWhB+YfAya9CztT{3cW&2(+>Qt@=ACi6Qz70>lqOYcU-@R6hUv5n&r z>5v;$j~b8#&gGEZHljDKhB*(j&GvVU?KXfILqr^ez62jiE&ga6G@E0}R7;N1%s?Bc zMThz2MyH;|0wV74ib)y51hSrsftS4D@L2>JH-S4A*F_#(q4?XiuebfdJ!VAfWI5c+ znDD?N>q^~KNpbvvMzpsyZ%HuK?d_Dz5m=L}3Bw2hHJ@V!=vRu#;*tV(J00R>qMnnJ zI?LhbetOhVFn?$;HI|imq+&#_oKp-kP>iQ|PTRn%mbAyS7ut=wks`h~W8EW}2wk7Y z&9I1hO5p5XdGv;0vnL=H7up{S=KtVPO-nU=kIoeRka_lC2*G0mR-;$-bqX%y2-l3= z;YW<8MmV)?9~N5cKd@1103515E-hc1r*(gmsA~TtQGM3}Br3r*Z!&fr8JmG~v|g^w z@3hU=hv_G4NGwL`)8ZBrt*<3;*a}!`JwHnkY0LbXa;VgO&U-{%Q$M$M)kbZiZiI+} zcEsvPWel;IR|4Nw7Kib{;12qbJylC|!Bb%jDfS7&9LYIwqw?C>iHumhe#SsbTV3Iw> z^m&*UvtOyvSO^zYY_|C9rsUMAMI9K`o|Ez)`oWOroAb6@!%I44N?L%LG4g5tSCFcO zXr%zo>B5ghT`@2{W-Yuv*Eb%E>v{)%wcBUxJl;b9yE~Oz} z;!)#Po@PnG+T76i3HIY@D!4F#CtH}RGL*jlqHh^cS^$b-v@&(2_S z4o*M9evoM_e!sA;yjhy$VOf(-ojqdcTKF0#5Mk1kOI7TUGNSAFi zQn~oD+z7Vpx77%hHK^3I@_g&l zzhZeNF|bdUaCwoy-$f+3W;uAGUKR4M^u#Vb$IqnQ-QJZ={_eZFlDR-H1zj3r)l?DG z21X+0P~dc8-=~epdNAeHB(I;#6ZT-P(;I)OQh?N^)hVF)%O!iTbk59^M;N~V*8);tCPB2CssX058nEGS;@2hD&*=p`FK>sKFtHlv0D#&r=+e!Fs>r&h` zxM`KOgveM!9v?m5&0G@e{=$Lg+EUl)tRm@UCbX-imVKA)*>S<3(Aa@f&o^9B$+T$& z`EFWr!ahE-EbGe)OCBzF^CKN(XBa0eD4M2SPg2~^_&RW09u$QXE`v3;bmC=@{W$4xPIx7 z=o-Ca!DJj!VK{Y2?yAsp8l*KNX=R=1{AAzWb(k4_BYpSaWoo70>X2LpTTaXL+@|mJ z+zeGusfQRE^Ymo!a8c~e`PnI&Kt4O`t(cgTPn}M-L^jAptzL7oyXTKj^yWO?>AHGn zAaQ2WF=_p#(14>?L0NhxYH!V=ha3jfTUw+rPTj7rRp|;wJVcf`{T4s0gQ}QI#h>He z1;CF@t-&=O_3tOw>>kOR+nG2*h?TiW{0jMT%nQ|9py`!)`qfa*19Wdj_CvBa0%bN2 zS@Y*5U2h8aEg3JWJSJ6Wu0PmQt)8iwteLrSIcjU{K*D1a>$OxlbVQe!b%7_{txQjIx6b*`}m+TG@0GbPggm3WzVm(7|gjmR4={m3LjOWiA>dn7C29fTfXTq;hHRLsSgo* zSX*#M6(nXKnK(>4Vmlj926FT* zIQETp!C6}HNm!V#gy+201_OE}WP!dXU099HY;dTQ%!VuwGstkaW16_4QR?F{4hILC z>`Mfk_$$Z{1+SugQM~Hn^?B!%S=eGcK9yMJ8Hqf9LR8Xm7V?ApYAoim3LD|U++Qk- zjw$YaCD)QEc&q#5)n{wQmUPJ_`|V!u*P1$X7A@VDJ1YaUJ`#gX=faFq5iw>|J)Wl0X4L?m$O~fg=_LPsfl&vkxl?d2;U(D-58#vap+vH#0G0pzsNjXMgquuDowIx~a=QCL<@@zJi8&)(j3NSUko+92chv?0HTdv$fj&lQfn71`;ExFZ%oUJ8`0u$g z1^wU5m1_!0VxFuF+Fg!)gq~hS*ezD$@?b#kPCrKX`rLB0lbibYr>-PNAhxI`C+TSV z^4H|}=NE0^Aat=lI>uR_OS&|G|MSb+^tH7wJ}*1o{jUI z|H?pmFu>Vx3I8Lt27@MIIK9>zk_z|7q2h|$nCtaBxc*l2;+B;=PF1@J z1r`np50z0tbYI;lEF@Rw+>d{ez@N@{q3!;N|gauAj!GR5Ck4Nx@ENJ4^Q4-+55{H?5l2p<>xf^v+eEN{)>SYm?ns2we=8zVwj06ZcL$ZT zafxGkXQ%7jJ~7kRW(Fn)sKx6S6n{!Jr8|nm4cs+2EycQtXOH6LQ|*1wDD&9yDL#MX zzI(L%evO^AwTPtT#}VYJDg2klj5V+JmK9IV`CdY8k|}6bVxa`p36~o23bLj;m}6kf zFQ&xJO1XSoqpg1@cM`4Q_p2+Wk#Rq<>E7ap=H3PpchR0F!U1T$3c0&EHRq4qkuz%b zJBiB_EH{|PCkmc_KWu6KzLMN|9tbpC&-W^@-wj1>NP-|SP(!|M(_`%ke`p=2`5vX) z=*S2ZdG$g4KnJeN!MezEyUnx^!w5~|@e(D`K{{xjJ?sHewEzYX^7*Eugl|W_F0^W$ zi{O4(uG`f0P`(tuF$lFE_6(jl6_fhPU3!JfLkQ&GeYCXh@b2Kz#CtjIIU8+qtGk_x4tKzO9Xvn4e6p z!7GEpr+4PHiMi}m^qnvp+(I=*Eo2#WBSbKw2oErLLTRhfm2v5anXJh%$&T*SY@DK! zUF+-~N!HYyw6wMTn%W~|wJ=V-x6XGRs0~2@QB23fEzbnB;q=+1ZHhxteJl6@FgR8F+~HnOT)0`Z#u1_t8kEP6$VDSSJa5CTX-1=DWGo$EYUN z*|;GhEPPLOz-V}7(EMexlxq5k=(6clNJ?OW-dko+BixG_PJ}_k%4E>Bz$kl@uc9i6 z(QUQks!EwLHDINgqpFBeC9eKuH(u?P9p_a0BzTE_ehzhUpSji!ksPWA8WDz(DPS*bt{eS2!q_3LD^*F@s7I5tIot$M-e2m)+3 zgqBTFqhj~A0^YsYs#?WXV=?J0qzZdqBt&V#^bGl`eu&_wyZ8?_s|IY#5@9ZSE6`u%kq9&;I zr8r#19v%?ny`t%{Qp}nvVdPJ^coiQKACP)_oV=Zq|oQB3ilSwdx-pJMs(p<*~DB9U-zNm6W z5C!rPp*Htd<-^YwrQ!EI71g3LOz_%citv{Q6e>ZYlfk+sG@5vmPq#-b*E8&hmW=F% zJc>k+1$Yu%hB#n*4I3MzpU!%S3%8vx8Wh=6$7`&Uj7u%I=~PoqyT5w->&&ktalfIr z6MMuHBgEaGN;=kQQFGF3dBb#S8`}rnS=sRV=5Ue5z%H}<$XO!hy9IDYS<@9uf%5g< zMk+hwW083~uDQ?o+lp6hmD|-gv2)Vn^N{))oTjQh6nu=u*l?}xli-I=+w?u4-#Qvq zY;>{q{j~yujnk1EOlA5YD6l38qpT{AY(2_jy9UtT@sl}BhW@&e92NOCAx@}TljiPR zZ7N^K+L+;^ul4l`c+*(hejKSoDmlX34rlKi7bUTtDJ9luVa3c)BTC-6bEimi-c9#? zu;^+?d8q+Asec7_Nnq(>R#iz#falddxdhky8Vytu3laKR*_zv-o!3LJEN(fVEHS(6 z!vpq7qVO%KBjRqeR=UH}(Fg}04}@pbr$B6;rrPm&3>OmwXqgzavy*T=VCPXz#ttkc zGS8e{$WpVEeJrXLmDvRm&+?{;(lZmkX7V8P{mRgUXpY;lNSAvmWL42^a zg|;$gzyzmi=fqg1r|PLD*Cm6**w?;Ug~*lbLB|6s{f7wI8bFN5c3+oTcZMigkWieivJ$O4nqhfch zY&bsJ$a`?*5xuU_R4jXIf0fyoZTE!d52l=#wE6&VNFp(Ual%iKn z%3DabTCHNn>j2eXQ*uiy--d_SpPijJ6q=7n>j$#4WFJ5!r5b2GCxHSeqm68dCOW3C zDMNfvCX`(g9p4)TOQ)@Yc5g{gQW8fxwMY%5p75qKrqO_6g%kZ|zz5p`|3H zY{%gfVv9x&CsNUN0q0X*)moR!+srgIvR;;Kmm=5gJ^ilZ&oahgk}w)bbgOeXb1jqU z?d#E8W~Anm<(_i7zRSxh`rJdd%f$Vx=||lnQlZAf1}OCMSLq$z;j`U*PtzyU%EiT{ zyO^zI`yS{%SL}rK(?={rkMMEru&lMF0w?-SOf830Nqk&XoQnoT`i^tb+4zb2E#BQl z6it^DSGHZIs~ht@To2N4RvnWjfp_P+W9!*;We*KhI=0%B@+nJ=VT$TL6LClcCsBw zAV=01Zw5;{zME2SCDyO56lC7VS#TRIUh0 zh)jmyVyDK~g&$?B)J0i0r5H!w*Kzi}jV(CM>i(EQac(ZpCoY%tNe)++)a0?yjxA$O zbH~v5wHzMzCtyF_Q`XbN;g2NM=FDPntuz@9N1wu)j0;4a)j>>UwhXls;@fqMz+s=l zC?n1sk_j~(0Fb#xA90Mv*fCZG4FwM{Ir=uXEw7!v@hw4M6B9KoiPHsc<~%%FDovY-Z=4s z;&{`JAs5D4+(=xvE$N7LR8actI5!{)Y{G-HY^L9%7u3)wBVMM2Q@vtDrh>XN;@549vj?yTquq*U^YW#i@LgeWO7^WER=i>*4uGwyIC43EX2UoAy)}rrZWc1EoKydDJmX5(9 zz=1T}t)>C81qgVPjxO^JNz)Pro~kVfow0l<$CX?+acPz6k}Yb!er_m$4SXPp9=RXV z$q=9xQ@YTS6%qrdVzM2%{ymVP3s1bYHD zC?l-{8ybz@J=P1*Q46`@x;vpPqv30WoNur^>2|KYyn~$VTPE|%6#t8PFn$8&#J(LT zV&Xnm9?q|6d*LQ~;!tgksP+|!LezJM$$y*t;3m(K5RB|9mxIMmygHJ9^QtcyB9~uk zms~+iw1?!8|9+Q0pXkRh4OT@4;3kW&$sY1wfE#;&SByDuZ(QCb{?o_O6kmGw^pDz@ za{L#M@&C(5ei<@xANm`^VoZH$NNNGlsOP7XgIlaN`AUv|!=l*c{P`Psxy(M&IVHGCblVN$|A#0&^@D~ zh!~Kyo(fK@fJEdJkL{k#fGg`uAj&`|MlxkK`J!@ik{z#I3;)dhA{PNcG^Z7M#t>ao z-t76Zb9#+D5&ya?NWo&hHnMvsEJximFW^kWIl;Y3q|v3VGj|4VHM?N7p$|K?+sRybcV6ICM^2MqF0F!XON!|3zCfdX`rXj>PDgIARcyT!2Ky5VF~T; zr7ez&6IZWemc6j2pZWvNIZgtb1JLnGk84_mCoGSSKf^fV!Y`S4j}btjo+m)g2V-08 zjAhq!d4|kW|Mly;&E3VepdFf0r}(h9@+Z`&bOn#kPe$0rW`E=9L0d8?QlRmgOm z>N^XmZMJ0L7DL%2sb}K!?}+$N*Lm2z3kp~!A$Cv418PA5Yq>;c63H0e8y5R87IkCS zqQ4l>i!*V(&b9QmSWc&QN2w>HoccSGBsi0KvxeY225Ytn>1&P)&kqLlrq571eeo0n z99fABW^OoI^uCyNL+1g|$ula0=*;0Tc;@Ek&xx?J!#vM7Z{9po1wp?zC`2&6Nhqc# zWS}+soxq|vWI>9t@u~w%Ad&D#G>H zC_An^k_5j7V-7Rw&rP$M%7v7DZ?jaa`+AE41ozmH{YWE_@ z(yv&jV*VQx;ym1RCt1o=OZhh@gdryN5G>`y7CS*-3+^}qk=ELXwN6i^ z%uMuM#LhSY-|1yZ?yhfSEXCws@s91K}0K3n59RvSX6HwThBQfxU&;{ehBcoAMM$=UgIt`L^UNH$DfdIY_fV#b6$qM^Mlgptte&hG3Tc)zX4shG|nY4{ehZ=L6g(> zK|oZZWEyj0ZAO-}Bf~tJWo3l*iO(ejZ}b?9_%>ao3$wuIyCVw>PVE8O9E9_dS4E<= z57lIy6_$FYS=Hl#whqMSa(;8x6q7sYH6XBG_{z(RLZHY@@H_3VCa=Z*!A{VyZ32pX z_SRYOXJ(!QUF-tzbn%l8--K5~LEu#npNq@X|2P*=aX`>;)})tgr}uD^k; zU|HYb#A@{skI`%ZpBXy(C$MJIF01lGnY-vr>|lO5wj!xWoW2=Fy!vAWn-*;MEwQT@ zrzm$J%-T6xHR&nH*~iv)hqlJqDW&k|V+O45SrJTQwsW6`*VK>5^}kJ6>XL9+Ovvkc z+cqAA+3UH*${W)wi~!zfs`SM!V`#v(8rr?MPFiX?L#OA~>u!vpeCLj`bP=E`<&&WK z%8&iCl75;AX;S1&x=r8IbvwK-Y|t#Y>6YVnB;wl+(68Av8T?%C^WxVLrlS1P$T~%U zKGYUjb8?dA%5m-C+XFbtSx?jMB7JZ!Q7-GR2E2kC8}d?5PJDW|VpEreN-9bYP>&+a zh3ltPCs?Kcd@NHJxc!bkj(?yJ{xds1FB=gzUR9~PCPoDczDd?rK_g?0%a6&(x9U-y ztGYOx_Xp~p3TXi9QGe>qqG-uP96|-fymE5z_?DpeKe+(59xrB_ZEbw2b5_-i;4iUl z$$zMjdR^0`9|uS;8%6sjhdleXbssFJlciF7^`$tlk6-IF)@sl+GY1C?eG+!f)+8Em zQW<@^JE4kj90GzgaB}Ppmgm;Na(x$gt3?_FEU^6}7N%^!espaJGUp_4!(4Z|P$eiO zr?+bgDc8!8Lc=`Tn?uL2K}Ig&jd9IS`#r?XgpYQp50^gS>uTIp))DB58EH(J4)87$ zx@3NNiXt%6n?ET5-Lg#T=7QA^Y0FmCt@bh0ggvhjTk-Nt>EFw_>ZuE)fI%&$)Ha6n zK30TG|>8m~)JQBmpbHfOgfx)<6nj@TH^4(MkEIwH>!=cWWO+igx?rMQV_ zh2oK#W-=AWtC?jJi3wX76wM;c`Px-N^`w6^ojGY^A#3VuJw=oz%D|oy^Cz;yeV*Xy zEK@+{Q?zd2()jtnbcZx!4$mI04a08mYosNY+qIw@jhThsjbss()=;X(rIyEeFZcdp zj1Iczi5sSi7q7gLL+tPa>8^1;c+;!~wm#^Cw-&tionWGE;z$EKS4aPVmMQ(uR()oM z@Roq*!2+QK!-aKVw7mGLDOQ?%1Tit&IGtuh$)i}(joFoM(vj94-p!unW~n8Gv9PpN zib@dOx0)~aK1={PM`q_Yy7B#bvuZ*E*7~#VaK2TL7DKACUp~AY{6rABR;d` zm(JVky=4`e*n1@c_RB_QJa&7Kx{e3U3$Yqf@72`Y@OS($H(K`!<$Zs{7OfL^Y5bLv zCqn5)$vh@Ay+W!QO?aLu}v5nP2MAfPp zH^h1LS|<{OM3(K>4$UguaB~&>uz>)e!48$^>CeH}8x8iZu%^sn?#PSvzzpunyJ^!> z-S|R9CH2@h;Gv_F)2d;pNAp`b=w}da_D!FyD%%x}zv39@6c#mf*T28je&6*}rmoFu zhz{BxDL6Q!@PT<;K1Wa*#!^SWej_)o&z+1YHrUN9_}N&_+G62EO|pz(M$YDIZwflr z{zAvmab?fS(!Nsr-OVgS4}ZKaHkXB8lwy|JzD5}ux^rTY=p&W49R+)+REUIzL-!>o z<^oiZfIRU+QzaF;yI~eRU#x7}W2zMNlN!liEZ}G)+ooABeILUBS?GH`pQ+j5(<<^L zua^uoPAM9sCb*AG9m1S1s`Cg)%vHG@TF-KmtbIi9dSD#Wc@^%PX5{fVs*p}n`(L99 z`V1?w6vcQOa7ODmk;1~AVfbNslLnR`K2m*XjiF~e4iF*i3!A?(WX@{qkNHYK^{I7n3GJ^^7hh%szRyIZO?X?Fpm*TdClD zm8a8n>&&()d&wn4B++_4eamPF)naesl`W0hUI;R6p2aySohI|0tJo>zy2+AD!)2i) zMfuNEvA6bio^B;g7w)UL#&RO8Epn0@9nzn0V>o2DB<96QJ?cGVIN>(pQDNqrNA%h^ z{b02`zYKJvT~W9|c4Kg7^r%rD&XqrZam+kBKpv4xxl%$x;W4w9;z`sK;$?%r9idOd&pZZKO0N;Z=3aOHM+e4phuhWsDufTjLV_CIr9&o+w-bhk@SwfpkN0gNSLHLXzEu~ChooYEfE#T- zfOPsyzKwey70kU>6>h{O0TUe$cto$0e}24 z@d|RVX$_CKftaf8zWxG0gW!mLzwdp1&=?DNfz-kunmv99jz_p}uo`yqmO2ldV%LMi z=Gs&5f|=#zxBgcA$QD^WywX3Hhkr1_O>ho%WOGU2@z=|JG|pkQN7Sti7=DB+u3O|j zkL3`rXL&mo;PhaaBK_Pk{+4_F z@x2%C21f&#$b%Gy)&GW&$iD`Fg!(hl@by33tKZ1o-!>FE*99Sw))BmJVG8rY13}FxfAbR=|`9{N5l=5Jc4w*dIk3p3;%7 z&Qg&8lp}a;l%i2|lW$P|?BeOP3uB0;TCZ%F8fvWno1o~K5JId|<;+wPiq6IqP+heg zw@eanU(**4_dod97RLB3P@^VZJSqKR^#qxVf)IW8pC9=(lv>~C)I6%3Jn9}P;i37M z@!DpwQ+xe*VAw-Dh(S5;3GUt|3zE6Hc`1(TEnpj2tqukt4czsn{g{D1C9RQXGi2{t z;*_}U>@ueE$m99p3G8Xh2Yt)$-@&cw|Q3wISgtH*~6K(=75vSA$} zc*Wimpq74j5Ek>i{$#eXW0m{L(pcL6F$B>n_=-2lnmL68@)b3Cp+WOV|7k#}Y3EF%Pu;dQI9nlU)>}w(cIKQbu1YJF$EO-f)37?O@l@+7}AY-O! z8>*;$+IBm0J(ANz=EzYm%Vfc#3XFTVOVfQcG*y!ri)QWQ5(b2I+>JY<~_^m(^fyCn0FT?^5Zn{s5JzgP$Z#<3W zw~^;tw0I$#!<27e*95jFqD|f5G=)60arKF7nUWYWo6shV>28kVav%A=&)KvDA!_CPO1ik57=)Cyz5 z*{(kv9J(nFjntKRmk(*6lkn2q9mu2mV@0X>%epT-Y4S!6c?Omp%2#@9mu9dLMDP_z z;l=o*rdp_-)bjrVD+#S#`afhP3dyt2pA~18C~GQY%0l^Uu3y7gPaK7dC(7Lirvhr z$TAp#yNh(9AX)JMtnVf+E;tG{lQs9X1p=C0PK*39LuG=33E2xsvV8$bXoU9oCznRB zoD@YiCtPkFM^FxH)SQ_dk3YoSis4#K!Ms1+oK@^oN2_0)gLWK@30WTZTJxh7c$o>jYD z6A9@GW4pBb@KSiv@NtmaWZyTV1@b)6!rAMklr#kCbS^f@0inGM_} z2>OG(pwwxdq3l{2oxj69qsq5-(Kk45xp=QR(u6=4-RlV2kY<_~>-+Y+B$cr8KU2aiY! ztSeCg0ZZ2oq>OAV#Y!#Ko3{nto>K)1gF0sh;wN~2c@_ghwn{2OV=mvA!?g&njb9_R%MI3mXS=o( z6qH;Qq{?QTF}=m4TWeFlm!HL7BB{n(hKOgy-6J4Ac- zd@I|gxold!04kY}wTpLFhrVIIj~}1*q}V#x5Xf~}<=E~d!YnnguU`Y{=arU8kdOig zeDACE_w>}sA`$d|%*Yqcpw^=|3tP!mVAx=C8RO(zLj3p;N_bX1VfWH$`>Wv*P*p9R>Riw z4AxVQKUGatR_zk5zBeg8KMv{K0+rSyB!gb4mv%r3YTVE@2t4>^`$L`)Q;Z#}+b3&xhcl=P44mYHj16U&3G^U2E4R~QEz z5ym?0;?;gRNdEpY@{En#^4)-1%U;$Gq?AZ;ZEw@O{TsTRi32wm*WD;ld%k(oc~KLG zSiU<%(=1utswBz!bn9b%A`#`vgZ&@Vpy#rXp;m>wI69TL&UULA%X&g`xq{C#RSOhk z(xO-lNe>HGaV$s_2-+QBt8YPm{uLNY5HVbaBSkjh^On3xXe;|RT<&-zCc0O5Y5KkK zdKW@z7V}fUn=-rFO702!-D9TdJ)G?A9A#7j%dHv*Ze~{e;1jmY4jRa*_!blIT42JP zi5%r@b)Sr}StH9EsmmeyHy~Hn`KzWxKu{Kws(vIXzw31q`jzwG(kZ>4SZI{XA$gQiK>np=#!`J&;kN3PF^G)n%>BqKbWq!B8`rpG$`U$xu2PL z#o;xqZ~)sn!EgimI>qU_>(OT21SiCAt#vKtw-Z;ivd;+j)=DG>{S`TkHL_d2M2 zXtZpVNgA?jrXS)WL2kWt6SbEATi{gj>MZI}v2YPbxGJR3ZA|)MR{m>{xiEPM1Xl*# z7bH$(ek?RFn#gbFHn3}`1P^(v>;bC0^*biv(I`5K@mw6p1xbkHA{pgOm0sm2p)-aL z79W@H)jTKS)kdmR+)_bEeSfeKk;O=>4`yLvw$QkA`U#HmRfQ?(h1FofmCM8*n9jkL zf9WdC#H=gJCh&q9Y=8txPb|!<9bXDhNlE;dXBWJHr0_@zK4rSPFr|iahDRT3VqY18 zwj(AeQ$6U07sd4uxq7Nn%+QnFco>JM1{ws;KH~-JhBNC&SooO+tXNijdqj zoCOCds!|Z%u5x7d)XnwLbeBP-gid}aX%0S4Z>D$Y6j;<8S_fz%D&$Ln`2vk)F_alN z@v*r5`fnW##*G>;dbOIqKsJ+pITKG2-N4q)kJ?$L$q#%?zSlI`pN5fDD|?P9E%juT zFx*>rI_Pmu^$+h5tv$rb)80raC2>|m&P@ykW$-l2TXv9x*;J=IK%Rs}os`}x%k9e8 zC#7Zip^Q$9YM^M2PG&LcMGUgXCIP788zud0pQgEt=n0d%Zp z?DcA#XmqxM){m66?$Tma?<>uoYfr$dplLQ;o=iR{ zC}K2g4^e|x6Ik~sF;#SmDOn^-GHk~xGq6j#nWo)?`tO0rvTbQGw|udMv?QrfxAkMY z(D%kpZqTv^)Nx(%M7M3|Oq8(HQ8DwJsGFkLogA4_kq-}4ZtEmZrvuhiYV)DgXl<49 znpDC-UL52_6QxkW@Q|3lu@SMsIwp{)cG-X1HW&qi`lQUTF_n#gH06@-D9DUDOoS6U z)`nB%Bl)Cn0sOkY)wPKdg_@$;FM1}ErA~aB1E^_248=+wPoLt?ts1c#>B=JsCt=*K zEyVj)IH&!hBq=^MfnSI~BIj;+-04i+7gv4>vuUQwd%P^U-E|d0U#JOJQ&hL^kF%!N z-$-RlW~H*3j4NuK!Y*5Bmwcg=o(O>b zYK67S_vc-{XkP@Zcr0@AwLReE;(D&seJ`5RiTE*@%wS2jN;TT?0f`?SbleiiAb>qs zm!-xLr(fX6c=_puaej-+YCI!7>Fj2D2q{;uTn+nw87q3ILu7{crBzyIx?n7u_ zc`;gwtjccU|F*U9Lk{#g&lLL7IVKt&-8%-2u~8@CQ=QA`NWjSYi%ZF5Dzu7W)WKB8 zdatFzb5o{%t*kH4Zp=Z3;q=wVX~$jUT-5=7Nh;A~YKn`pg1I@|0Q?s3F zA}$?-okhkspN0LFiMV)pW~)%i(mj@}8U$Rq$)H2!^@WHU zWTKW+fL9w20|d3G<#GP}6ivyq4b(bKn;!p1$n(BoKq}^(WFTYU>yiT%J_DOe47)xi zjbi4jsqqi$AUCNy7XeMKsmTor67 z0A}l#Qa(0|W_C)(^xGKTE@=Hi=d-PgV89l+n)=V^5Lw9(8` zPK||nMpz9LJ9h84brEOQLpcV;kyS;}$x?!96p{$b(_fkn$MA64tssFS;QpRLD;|my zCbP$6d%kqL|3q&!!2#u-?YNWMchvlI0U!?rlEDi%wbkPd}~mGv}AUhFA`@@x!UBijHT@0tWVRd z8iu0SAC?F;YUe{69$4t9j_;fCko&7Cxz^}oyPw;n@YBu)X1iC}b~|oPPN?_nJW>IBWbd23vrMX^HWo{t~b+cu_# zEJ(KGvGm^S1nV>rF%hIod_>BExA0q+XI@8QReN(v>YWu`Lk3PBUzOmT3Luz&G&8Q9P@Gh z{rjafWaJ37GZ1en-xL zu~zzrcMkO}kt%x{IvGURbMoGMf~=N7{tHhTnC^pEIP%BBBb24oTDMh-idwj%A43+u zCr7^Gyhn$gC|d2C)`w_!)48$Gi5vT%JtAb~tolWFJd-nQBQqmJ(A?`dXJ#QFYH1)5 zi&*id9ULUTd0B*A?%)QIFu@n--~<(LI?C+?{VRg zVC%Y)eNH9`p)W8om}T0yY}ZVSA)sRTiMw486rSUKf_Gb6k4`J$zK+6#{U(`1_hNWF zL@I5PQ7G$sr*eEdw?1A?KFub^-gU$a{Sm@7e5k=?PLd288@&|2)dAwS2nwj?Ygcf! zmfy*uT4H)QyksaaoSHINH4tA&(W@1Y&+$Pc6p!!z5xce4N>*$pGCLGWP#`55__m`n zbon*KhR^Ep+*pTNiNZ8f%yv~02{bO{qv{RCVp?U!3^vW(WvKB&`;FdWFO)z*P=T{Q zGBp;^ulYFl&xo=5Z3?w6vnk{<<}z+(dU{_}bqlLvTMXv&VX0}!+W@3A4OOIOj+}3G zR#;GkwAyNvEU5LTZ>KYa^Se1$ZY$k+o!b6fU@CU0zei?Bf3t+c1vUgt^%2}lpVvy) zb|gA=ISj`1%ho6%%owB-2KITL>$0j#hScsn=cMGWMmx%{u=g&-zr|1McP{`H#6V>p z=*r`QUz5q~*S<)2($Jmz7{b@kPXePpvR&>M&}$AK-N*pDoI+eGyXl3>WFob>sc)UI zbNf>C9Z32hCz$`REj3NPMy2L~<}T&%1JBYLhjY&kwVJmMla{q?1w;m2DyjM6n@d>O ztyV2~cDAf=4X#86QnP$k?-^&{SokhGnVjqwTRC^5Je134J$0*CUDhdq7Yd`WB~5~Z zs08u(@PWcY3+dk)YSsPjjCps#(BPt*ON*FIT5fAWDE=ua-SWkU90Ze`>7F30nKefHidkEF2G`H zFP7-xk--)ppmcp`L@QxyeZ5JopR-k`ls0xR*n!=Uxw_P|BBOG;rYNIwmm|xc>*Tce zoxejOLe2ybTS)QO2iB$d-fSjl&bQ^79iF;mj%`I~3O%e4wO6lkA^`+BQzlNaPY61y zUZCe06(G~BR_@azA9Rr>10th4()$$1POzoIR~YoPQpy=56ud^d-s(t%-Wge0<#|4H z-Gvplyz9&vfya>$2eQq7&{-XGiO1CWTXI#z`|Wo+#vT4Y=vb?B)JXqkE61F}-*oKU zLEpqqrD{15d%sXFdBbch8F7I=`mI2Jf01El^jE4IN}n|+j!WHmaFB1{3_{;ACz*Ff zJ5BdxhaAJFTrUB#dvvFPsu>`Z>=g(l`zn#8NHg|VESa@<5C|uJGHS1;{|1j{&(UuV z;IVXS0}4AQlPMs$Lfg?F))OBGc{V*m(9)0Xw;+$(MJ#RFUdbra4GII(?PWa;1%w@% zveTSL;XKvLar7};ui>94GdgRi{w}-Oc&ll-`m9*dt}I*zA<9ZavaR_H5f{BCDb{Ia zPplF0oaU2>qk%<^E6``=)!wMC-(I+Jh8X;^sRVYQf{^~a2ZsUp)iWd?Uw42cFLw>l zvi;!EuXw25Mj2$}{59n|E|*eL;!G7~$U-5SeD@{q<(jU|(Kgi$E#griX1oKLe=*x97A(URrzcRMT!4+y7Gu5Yv$Y`fczsrni?9Za@Q1)io;+v+|k*znD@QVC#<$Q7g+LdZR4gtw`um}~?A|_y7>cW)=2;(qTIFwes48EDN zk(h;d-mY{@jdcn~qg3kdYw%I8eqBW~TOa}(E&UO*)~ZUIrB*!jxK*t+!F>H%bCUac z93d+z;uLB2YB1d>0y0{<^}0fLFIiOe37S+?;cH1Y`;&yMEuNUZ8)PAG^VVwV6E`{;AZube=1MwX`yovu zwz!a7cb<#fchTOf^TLM(u`!8|L}0GYc%c|qeZfcaRqu!Ru{mWSU&N^i7?H8=3h6?B z+gZsQuBU2ij~l61932eyWiY7n4rNBfCXi!)=6J<-%x6vul33}!p+LXd3X zp=@q2zko|4RipJKjhUS+UE(F_8q zCzlQ1Ryts5nX9KT6Ip2gB6xkPj^f3$mB@J}X&|~YGJCexlaJGNPAl^-g0}>FwwXpg zb&O$R`w05V9~9kgTIG-Ff%1hQc_;(!=tc`-OZ!Vdn9-=I43o4+^C@l;Rr4v#-WWm8 z@tO4*l2LYl<%dhNx~9QpPerGI;;8S@klwHaZ@fZxGG{}Qs&PKQ_(&RJ;+NDa{z!$r zr!p>w>2{*}t$Mrk z#T)TK0#88Mc-xzb2x+&;F+br%nVlYuhAvIQr0Akco*)NwMQNQO)XV}WbHkON`==xG zOaj5U@|NLsgbRhzoFj|VS#Szatnt|6m?xd7$2#@nKsmCar6TJwF{QWm_P#av6HikG zyjaO>=xYm!Ly(Bk2#;4Ph{OICntr1o2V!?HGU2#iFrkoEO%AkZ72P9Ti33pi?~`K5 zEwG)vuil(Bv{YXLOYKH?Cx3|1T!}eZ)1D^3fmt4OL(U?Jm7t;9aeb;J)-gpkf9cU0 z?W1%dU)?m-v^c0&(SrG#)O>UY;P|L0r(ykGC66zV1!A#Q9+r&Br)WjB}b2BZJ zIMY+QVs9c|W#|Z`Om~WXPX;R(KC8DpGBCDLH@Wdi)!+ z`uEFKaJ9AjX)t8|{ja~jYa|S$A%(pmu9*MDW7MVtFsNkY`0Bp^j?ZW>xMyhWoXzhy z{An^>`gt!AlrFfZcs=&N|Mj zkMeVpgHL^ssAUP_IIY+dY=@s*kK}?*EyN_~$|l>EIhKHmD0l7Gj*%5<6dSDyU7A(x3iVmWHp|P>XS4TJMr*f!3>0mEV%JDfYa-P%vR6s)prfgiF z62oXc-x|ON%5m+4`qLNp)JQP)gS3Z^79#}gl>+ADm4Q3@wCVeHLT9eXmebRkAPetJ zV8jmS^Bf)~2p1?>+KLBxp&UGKExb~|9m#I)8P+kX^Ep3n&P3^udg>zD$#%JSrq1g` z28zvid1iufeUr9!`vI8CA+_5J1t*LMEFy6+Y@5Gy*$;!!kT1itLRq<4Y-0Jk&YH8c z`a|4Igm7Rd$=8*&0p*?#2+E=G&xqna=Sc@q{ym%0k2FonSV)DgBgYAsYQ#a--MY-cMeoQpu1uy?*)LU_ID$@EuJm!6PGd%(DtI zX=!PiT2-xZqp?=e37*jO@x=f4i}*w%m%G*8YC8%KY(;7@q(VOKCu}R0u`5Ng*}PfXBBFV{isd$&qlKg zsj5ZgE7+n)lAk{FeZc@}a|J;}y&_MdtN|VAf9|*7h;9K=}7h!MRJPAG}jYA-tYoc(}FnK&N-T%YhTSrCRegC71frx^l zq9`pPEg;gVba#VNBQ4!w3(_$(C_SWfhk|rS3_T#t3>`zqJ)@7$BRrq)`mB4`y6d;r zz2ERZeexI|?K6}6RYwwN0xb6%5gK-Hf4jQ@Y;`p^xU22>v&E{Det@r>F&KrzV z>a#B>@wa6G3E&$kA;v@bqHc&B=`RtO2m=SlGL47a%Ru9nu)6k`JmrHKNz~sBUp16N);O zVlD1<_FcW41D{);fsq1H1*l^r+PP%?DJ)uKdu9C5bA_I{7Mj!MwZiALf7LO1(D+Xs zBU!W|U2nSHopAc9*U~NNy9`AtLM7#RYkGJ%T|KvF6$dNcnnpRK>>n>~lPvMO&=BC1 z$^?5EB@EmHHKZi!|2!q=bjd#nE4svEvd2DICw#epk6+h0?DbwfS-7gfm8rXI9a7H= zaTQ`?JUDQjyChHaq`b70oMvWK3rZ2#N9$nIZs8g==sCjECO714D1Yb7nbaha9jS7t zJhZKcDDf@5dhWIqZzTjwk(Bu3P{}r``W- z$$q=0uPxG&$5zI|F-L}$7;vgI8Ual8;YFQm6K|Dd0Y)v^KmS%3N6i#M5H`j2I)D_?cX zbbQztxmss`K%&2#$oA~0j(54*Uxt3(0TFGjnAwoI8|&^?5LlyUjjmc%R!CWEbuO^x ztinV2!PXs8bMweg@JFsQ1V!c<8sy}i|9KQ(g0(%trIUcR-r=vV_{Rw^geyU49MwI& z@@GUnb_%Ed;8${;Y~}jfGa%k}->pF;T-6mD5TVgeG5C`G{p*TEene=`!su#v zw+ReSdU}Es2z}2h0U@7|l=0(RbT@1-JYRSs_wo56UtlE9wTk0h|D9rF4^l-8*6TF)-SglO+M*QB#5EPtlJDOIq|poHsXM#PhU;1# zGjvX!%L2Kh&6qP*?Z}yG_w1P@pVY}We|(2dR4DbE>>RD_bae8-?j~t^8dSHa!ToY) zr3>zpZZ~BcDcPgjSGXEb*;KU;)nyT=k9|4B-)kyZudzSQM$pY$nt+VEGp6BW;G(CZ z0g(m2TB0+u^$Y*W>}!oJc}{Z(=uz?9b{GHRM@?=nDqLq&xCMYpxH4*{Eecy;k>{H> zw(9wH-2+!le*Fd9nxsB@K#)rLj|8cpVuTt|;+u;%`evuKgDFL?65TL$w>E_=-pnM| ze=FPfkc4-fgbVgD!x~%gMybVFz?8vcrSEU_;LiMipMh>Xj1`tio8k znmWx2s)vPPXlMx=>}!=BAe+B(%0>U5WBUcwwa<|3?^G30lA>(zc~E-m2OuDcU>U!a z#d^V>*4{I(6##PY7XmNI|Q2tKbIbL%jlfuW3^!;+@m=5xg6}gyT(qsBG+?yQ+TY_GRDlj&lGaJcT9|{$|Q1ZM| z97E!^A;wvcB!u2%GP>+?K%>Lm&|Ff^Rw0+J(;*-8aS=Ew*i$ z)I5tXO~W_7oP+P?<-d9NZr;>|npYjGvcIGE zzSxZ)VSgbap6fJm=Z4asDwEt|`k8g-ce%Vhe$^*$WCwTl`_|&pK;^VHNpB3V8!X7 zL$I%#f)$eA99F`{kz*9oLI{eKo@6u~*iX}@YkbP@7!5ac`YPh!t-hI*xHP_B#@>|O zM;%V*XdGJH0y^ z8@(zjbnfc6x4QZ_PBV~P4;~enJ9~kNYCJB=xskf8LdUjDXK%6bxm!g6d%+kj1zu;n z#hx2aOVuJTjLI^E(cQY5d#im*Kf{n@`ru7M)a*M{RBdi0pRgYBp)YstwG{1=1H-s; z=6FLpk7%KT|1^`p`~?az$~MIKab^qR#gF3n1kg>}tLU*67iv+?zfRi|KDZ!Z5vAcH z7{n_*^aWXgRX^xt7&^>nfUMr>@G zTSP>Jx?H1BbFF{V$Px9upF2apQ-CQsq-}Gk*s^o5 zI^IY7vk(P${BuU#ysgn4m14^=N|pfP=%v!?G$m?j*x+a6(UA&S|LS|*&tqBwsTwwt z<&j1pR~A42uILA`*75CEa@4o1d8WsWm@lrlG2UeLD5alRsQc19opW_w=RCR#^}ZnR ze8U+8Wnb9`Gi&0R^m7d{25P(pmB=~XdZ)k&05tTvnfUR9jC1|bPLGt}IR{-vOuzJo ziHt95auD_?qI1b4*Hhd_l3#m$+h+qu9H9qZXqp7>wP}$!$Kq6MX9mqU%(v**?^oGa zf0~q-Xw)#T%=j)lcQ>AmMiedSG4hZuBhXl;7-e|8t-Uwr*#-YPV)U!k)W|%VNm=nR ziWe@i_0OI(gjb0C+5L|etNA92uU@`$cUb?_%J@csNSsas-Jh-3Ff9UOFWw)uZ<$1g zQi)ZOV(}d7_UFR%*{ppzv1Z`7P@KKeBzw9crVJLf=kR96Grafx?>-5Cv=%xx(1mU2 zDIdi#U+gO3KcDdAvtVw6e2N6Zkiq zcKyv|%Fd&2w}PVX$r8c$e0V?Wiv^I<;NZ}*!N2_72Ilviuz|H`yZ6=Y&!-1dB97(4 zW@tY#K-8W71->@F%#aKyDs1ip&w3KxzrWyu{H9J#Pp<^p5-Mk?(H91*=fP%%p_VV0Pjjg4cHP8 z64E?MaMN*IM$q|qU`xmJEmA#rAfu$L9BUt=d}1!YFxNV>57e9~^*Z(%MqJ1Gc3&WWB40b{qc|)k9YN2u<+Oa)yzh`=wj&`}+J2sBsVO`k%0F<_uPngpla$zgd$xUp&j7})lPPwOK zp0cl+Bw++aV$;DSIhNa~nMGv1>$yKifptg|MC;RK8fqdPO!>{IsVA`I&AOx#AlMhZ zUOJ~)Jwc1?$_~6l`aor-lH-xq18XKo2aS|cw@XF9F4@SoIA_I!({}d{h1_dLa5*P5 zyt#ObaSGY%dA?Jc9eKVl@Q4uNlgc{7nlW+>>yV|^x#!3*Qr)R`Ij=Hg(~7r#u;S2V zkSDK0+H$!_G`0$o(9d*jzLGGF|LjY3u&`YWnQ9TfHyG|b<$qFSvA4M#ei*nJl3R12 zqcu9HW+D^AKu1T%X0w(FrK~)uHGU{~uckh_Eq6ld2d=I-8`uvAc%>{N+7}+gIX-L1 zl4LXAeBk5nA6{DOf!At-b!-X2F4P7~TYP)r#J)13z7nnaqLM2iI$B2Q0n10zGsk{+ zferjhnDDvPKRur?oBcWEk83`Pv%C|4|IIR|yX6w~qK*?;P7*;J@y=-u3CZ z;_nat?Du$L|NOl49>$^)L(n82>%xgS3Qi8a;&7TEshX5x_uYRQ=FiWpaD#3+!~|VD zIl{?NYx~bh`0cvBo+JBj?_J6yWYB9VLVt3Ee?6xDO>$P${MAIS6Kd#Rm;XDF|61T< zll#9Dc|u8?xcpy<{I`bqLxBG8Z0##cj+d>}zJJs+B>L|4{yH52wGH-WYg1Pp6~*}L zm?B&5y^p zgR`<1zO(E4OeZvL*?;${P_5HBkwil&UE98@ki3-Xspe}PzEj&eX?x|;fmR;LkmbFp z0UyKeB0KqYc(tLqq!dpAY_y3|l*)H7IZDTNvX%+HH!H@```h>wCSO^vDdxuHC04vOX_!uk7kCao__+aL;7@18*Ur3#Gtkv`^C0 zm2%qgO^`9LU-|fnZ8n>zJJH3Q+wNhPm6FPs_aQdtVOP82Jp?Z)g1ni>P^CXCq7|tH zZTLJQtXH&{?TGd`f8xDU&uzzrr~W7tTs*M&M)rP7=T)yCIFqMBPEEG!yli`8cFX#F z<5BLM^$oY9v7noQs8ux}A`t6h62DO}f$o8(FMe$A4fj{q5Lu^sEU(gHU2v8o&wBEu z>1sy?3-JxQo6JV>Rmg?g>Yday@len|qYT-g4LSu@Pu{rFX9V~E)$W&EejyS7k zt<9WbkRdgWB}!fXB*gu*yDc+3)`8>al9>fRvEaS6JQ$OQC${{g5>nl`9hXvr%~$2J z4r%S#5<9p+{zLNxARVm&@;OeMNbFd{zO-AQYR z>wbEIc$>9&gd(q4;_N1a4c@}`&?re>G5iV8U;C24A{7*Ewey(Xik!9ZfSF7zNO!GR z=)W(5czC2ZPAo74ueb}oFPil?J$?EtGw$|dgAH4!0bG;Q@nm<1yZ)yq0-IUk+_o1~ z$lqdjkFo>FkF`URhc`8+SIs%$K1lg}>%GHA@yNYLwIyeP38uz3clo%I{nMQLWji?w z0fQ1-0Y-34>HeYef2&t_Y4AnsZ?8`JmM0ES;J#s>{mQREt9rgv$O4()U8QX;DJd81 zqudZ#QI+Z7R<<=75Vf?wHRcuCKNcH;bUtvsoFbjJXFSc)I`tt}fPFUMHk*uDe@+Z+ zVZ^1tc&8*B5n++8*Vc3#Khka_G8e>>nCw|J10OYGR*Fs@5=SOcWumM@R6Wd-BD zb#98qhEFbAY13_O@)MaO*xhRC*f%3JAlJwFF|+QSiM4b{+Os&Yr~A#h2Yz8nmRtUt z=2rZw$!Ps|xEddkc|%N&*q8O`5r6hc+0Mp$C0Z*+Sq@~22Y8(WP7N_cqRv2M2XBzG zM(gI0vj{qLTYz@=Qc0hkmaQ@ya`<^gX@!}zovZJgxF^~e17S7&o= zhqK#L>Tgg_BO%zS^VeQmJC)jWFu!F7{?km;`8t-E`hi+P75P#-4RR|(Do$g$==ymXQYi_Gyd5BY|n!R6<} zyp(;vJu@TmTEv)Pho?I0Xqy}rpLSoUO&>p`5;?eSoaQLs5WQ;RNI5>3ztj-5nhKc= zjVW?t>+A4*H#~Y~Qdax&=2gs*NF`=Re>1s%aUf52rrk@qy^lCn?niD-{|A8A;iHK@ zn!HJA8D6fYNRbYQdU{~DY)j@!NlE!;X|BIxbUz}A;;@WQDRntG+{>MEi@D35)}dMu zJTX0%9n%B18E$uy0EKZbc`VrsRyn0nZIUorYL$e3|9-zLO&7b&c{P!b?Qr(A5>ASTt=7=-oul;_oW=FoWpz_|Dg9XlhA}-hgNB6$tN()mjSq zhTV=CORC4{opJG{h^Z+fB6hQo2UhkRd7DDS1sUB4E~okK*8Ky^oeI?wr$&-&nF6Mz zXY(t|Qz6t+8b^n-5T49?(;Y6q=JoKfBW_y}7f9@t1&qMjZC3cMrY^fXxv+KkaeB~p zl+(uhaZJjN`4m;ym{cNp^V%}NCq)HgQi@UEiF|o;={m5%ASLBUU{z_GeEcY-^OBj=R zNDdEOMhW@|o%T{|l$EEvl(OLh^WwtqBmzV#*&>^31%K$s^=-8oYUf8YH$G%fUVOz z-faJHfaDR#ryY#&?D0l%X*aJ~G=}CmZt})z%xb6_ayw17IkQ_&PRp79o)m=7?S4vcZ+_rcZ>n(3K!x@y zcqJbnUO-!PnWn~b%pZoi@fL?!(o@W!`fA3=i+arx_czv^qEMjDXpqi6OIM;i>Ap1# z0iZ1E8Z&j4F0(-b0s<9+x7Kza#I34l9&`W>c=&Slkrs$HGb++Te?#PrS<4U5UQWFVSce49M)9qXxDYchy zryg{Gg3V^j7Nb&qIoieqiH!jy!8hNgP>Wf*Nyb`oR8}PPXv|Ny6z^APmfNE!mfQ3x zsinf$TJus(oHy_1ZCbfAWmgc)@)X@^`uIXVbFikSwSUahirIlsP#Uod*W^B2Dh^-U z_mneSH0yj{l&KW<)3Ips@n)!LjD@#4_e2FO$F`cr0x(VPlJ26UoMF3jkLuI0d$UrQ z&&0Au7DX{XuJGg34NoM-y1;b}U=}j(>;?%-cT$G|Fr2tHGom?I-DfL&RQ*b3&FD+w z4)hUH51&Nf-uU>qk`7Z$W(fuy%yrvb3=XB0muO~?ir(4{%uvkBryMkQUvj2q9eSN!MWZF0AtA@Eo}1)d|7htoa|Dt$!9}Hft6bf66A3z( z$A!IqJsGtG?0CxN^)os3nn!*CuBKTFqeYLz0z5m0#38E%gi5>}Evrr?* zi16e{DgF|H3ap%c($)+)42tSuYC zlt{Z$_LT?j4ifCnDjqS)#jt+a9eH99H++#cHayn*lwbTCwKhRz*93QJaQyX6Etl zi>zW+Q!}qN#n^HKQ>eYtN8YM<0;Vy%35@jY^LrqyF^omg#hMZ;;GJh=VOqdvYQ-tz z@rC-zRYjo#m8u2uq?!+n&Mq*)lWBbp2g?PKc` z&az8v;<#`YD%6$OWy^K=u_p$^3B%rnGG8>>Cz+ZsM*13Oy&5w_I=fHEHrz6X(_!|= z&eG08!_a7)QP!&jQ#YwmekiM0hwJ1$PonwfDsufmbwquJbQ=39%W1W00CUravY-5l z23C$ACkcYpM^T)O7&tt?yv$#4Yg3*#!x>eAB<`7Mkv~5dB7b&Hhsc`e>o`%;zhfr< z4yv8LD`fECiAb~mzwGuCz5$5B5d42=#2{mXjqVf`oXeMSBk#*vbsW>WywkxJYWK5$}-k$>N$Rl zT8}{;1abwjJsLIM(HR7lk|sG)06G>g6f zx7T4Tz#291)_qS*xDd??;2_WeK9EkOGi&SAy~%bNcj6PC_;d20xYg8G^i-{MA6--L zNh*rFpVSneZ+s_9hZ{=pL)**@E2F}%01N%t6f68>p}(w90h(S)7!+ z=Tu-r$6vvkEVTRj2I*r_VKNC7E7C)ib};yN0k8qI*AJSwV5RORnf(j-dBJ>zbdIy> z;8(m59HgR&zEVB}DO7xX2{0{u+s0}((VWWsavtbrCK@zlWIKxEoKBJ-jli%4;`vF# zdVJE7rMD{VDtLQOLVWmhxLskvbAgFO*%=MP8Uf$Vnm1m1|mN(LuH9FZUGua_XTmg7BAmIurDF1-8?PIAJsvCd6}2Vav>PVhL)vl#d+M(9;~za%DA;Fq|$Yk3nT36PNT{G5}Z zYp1J>EXT0VH@)?rPu^AKfTx*Aw~SU*7?0GePb~~SR!)hZ9?WcfWGg#qPEExq)0eA% zw_sML;9&&Z#ay_k2Y>)=!_;IoEFTDKEH@kdqi8sKX|0z_tKJavyJ{pR87AAN(D&Kx zPJIWCB#6flv^D$CyN2M2~NTrgJZF3?Uh; z{leTy4L+cSH?#mkFdcW!TJma9+XCWfp8V;jr)Cdl8`l=M^fk5T_kI|5f7E*m@FbgBMioNEl17o;`h|7~SrA@mk z{fUOH5Y?hI@*eFYC6+D#+rhl~3O45axI(@AGoBN~1eQku$YK=Bfe+|&q5um?F-Fx7 zu05%-O3m?^G-(Nk=j8WkSL>YPxMFoL@Y2F$x6Aw+Gg@Y=8o+&*UQ#?WdsiGUN`6uR zy$=HZU|%S)XwS3#T;8Z>$Azgf>~Q@vSBkG;u>6A9_!AV*X=V1 z6kx0OQw}8IrN;r@C!Ahgt5dT9_U%I&zgMB*yCqw%t-`!~$1cf!P#sw!7n?^j_bK+T zxAM9wv8S%-g=G3m!IR&?0(}yK19XR4KCoO?Z5^SJrbAnV10qvd3UZN#E+b)fZ9DW} z8aaC&_F_pCrtm3*I|9Drr|XO^ikR}-HOeF1zysAD8C{N~yI!@uG`nJEy*F0XKAq!c z#p%)WO0~*k2M(D*!)V_|CC_n=c~C44aYlHRSSc>o!PlhDw7DD}Kl83M@t=;HyYnD9 zfhYmVnkaV>)tEHLIvm(q&3PFySXs3D+&uC+otjJl2`}ZAqZu{At(X(1yTLz4vHx{T zgg#5S$KF=-wAonv-U}i!QpHScro-niiVDWZKXvEGx}iS}1+1;T&ok4}Ld3=#b;PjE zbwd$7-%@sMCz+y~QJl zU~rlHwET=GnqQFJ;xor8uknzEvUZ?VRr@^d-S}W;#IhQ@MQ76udQ;Z}2U+%qt$$I- z-1Pu5-}&x6-p@00O6A8~aGk9jHEDm=p7KQRy2{3>lAXou=^^tRc2%n#rvi)N!(5Be zO37lYgGOTW?y0woosfkhzHVKBOGFCi9C8bf@o8#lMPSJFDtZueEz&RPmn3nwakR#u4>@I+lm z&@H&Be$)8tpZedChGV;Kg}%L<^UzQwF06M^eUC+Q_uI{cre^I8TFXHdZ{$>t+;&6#^$39~GD3blhe9O&R)G7sYR}9~R9!`8|Nhf>uZ+Eq z*P8gt(X2R#pd!ubQi3|*0?p($k3)BH=?S@ zg0{C?*tL}`SpOmnk~7!dk_JbLrBy52EEFI+}*$rH$$fuyx66)u?J7k z&{oogGPFYa9Njz`hLvR6Z^IeRlW<*03-LE4vnUGl5xm7BY33JEBXqh8r{81GNn0*T zy(**&TbeNHAW0m*c?g{eFJ1raVomgJX|$lnwSC79lJZ$AN`jWLfXBK@VzK@@12a6c zKddfe$tScil7(NnC|9bb52hGA1~p~ZitJ=8dBW2wXZ$jv+nj_kz0k!(z6ub za5ieOdAKS0)@Tp1M*a$ZL_K?&Z1^tseOJjb_no!oQk{kDdAD%f;>a zAh7bMuM+eQi!5HCf!#4&YRUSMC$e~dD#>K2=Gx}+~^Fx*vVYoX`vRElD! zW+yFeT1NQW8O^cbCD4sX=i<$Puo!DAnmb2Y)^eHzxiv>SnIrw5GJDo{HPfK7w4(3$ zdAt2shMg=@8scI38?AxUhotzCsvl7O)<$$@9*cR}IFH6(Z*>U|Zdi;HK`POU6rcjcz8B3@Qi z?+ap0qHW$J(FUR4eWfxbRpl9{BKP3MRO}!Yfg@kZWEA8h>MZQAdw4_hnlT(T_A{iz zE`K>fHs)da3#Y)G(!`qr5iF1o|mPVvMbLF&c17kRJ;Z!DD9VIDjHc>H56x2pL()fN4@MLmsR|1 zRLq|?Q*y_4y1i7fQ-t5`p<+IdwRX0i*w#xexJ0Lphu4?1d|iJ{>2P|(kVIFSd3-@n zhALM9qpIWv)uQAB1WEfIK!;ksr*%V5^-1pp5b=rxkO`VVCv@*0JsM0V&d6L7{XiHF zQNeNmMA7XmpDMd3XF5?DKptnDzxjq6FOTvA6W7Ti^XYMxj1al{ zS!NdOOo%9oiJHR_2hkf=sP4fC0xm>jDapy{v#%aL^!bW=Gh~C#G?_GwZ6PX{Cb(=Z zBB9+^aLj!#g`0DuxsWQ!WpXNzEW!iQ5L7OK2DXhb`u)wg>m(1Rp4zFZ%`&NKFO7fl zJRB6z`YE z^1}`*TWf^+%&s7_+txye_<8;(@eJjHRtX4C=sm@5zB4mZuuyKEgjLD|$?^L7o|$V^ zM4iU7Rq>h`31j(?POGj0rN7c0RybcIiY84z-xp3p0Qed}ct3T30KGzd}K&bWtmy|E+4xjDR=bpVblzYPme6oN}=X>JIXm3@Pz1CmH%HIm;hc;Hc+$9rL zFSS{AGvgBXhIyuC^tz~BCbM+hUrg7aek{h6nEr{vpU%b)n->L;&gj`U@e*81tHsZb zTl#;PJ+*>oBUw>oHi;2G-*fQFw#zsC|G@MA=O>>p+&Ao{{K9ZdIsOMv`D_+&sl6{V zah`aWe}9z_*x`gYdM6Nw6Fx4`VNU>M^t{EhGS>g*&3^`Nf04w>N$}+F>HPf&rx`Ff z=c88t|4Nh<1=f@3OZESD;|m(#RZid0vib@3|BsUn{=o>eGZVqTSjxYj?DxSVRC_V# z*5!fkwuZky`PWPQ^^Ye2Uo#8-al(W6_ak7bdDlOyeG9#F!Vddqfq(ynfnz?L&HZmT zzVaEo%9)3SBe(vWL}&oMi58Xf3Agk2?f#zI7v6x*aQpM8>4txaz=_wXr3BRA|3jkq z^~`2R>7SBlD@k5#uL;=kB~)q2P4&%>HKYM1Z#sZ}0tDx$~2xl5A6FK3$Eco8n$} zG5=`~Pt;KdF}FLHfbtG76{Y)`tCBqtZ}S0nL$lK8Q`Xq$hA?x58bE5#IMN)z*KAXpWeAnTQWJJeSD<(eEnk!hOw`@knpqfs}U%nN5o5QSib5YxPJaH%A zVOTIVem>Wno@B@=ID}fdIhw<(Y2RsfV9_wp3SXjNI9hwSk5`_LLnt7Diw2E)pl~ox zccVo;43K470J~;-ZuM@eMI#;}#d6+}fG}SEp_LA8u0L5+PJkNIP>XK%a`4rbhIs4p zuRqyv|16rjjQ1)?OCsBYUfN!IeS`59_t!yP!T{eekh;jr@D+r9|6Y46wI`@#&SXd+ zc8bBb76L*7dVSA=-Vt*Zqp(++47VW-2%jN@YSbijX1i; zqv_V#Ueel_z)Yxnl)PFn)r$n?v%Un4q znutiCrbE7Li`<2ROk4%E>_dbGJWEr%`e@v|tlZ`r(Z+zX?bHfI=&kC_==qe_8Iys6 zUg6rRWHj=TO#K{XioS#&pvyH!S_E?OO})(6m9z>%-!^0n5_|A46uCOdzbEp=shdwK z>ApI%*VZ)i+q+>RMI7yZCn7~cI2L1hs({b(h!lC`%oAu`b3~8Wi&7p9dGu}qV>z2a zXRk;q7D$s3ZFm>Stn+GVZZYxt-`2SsO#fwfCi4(?XE7SOt8#X?4v6=dAfAGCP6YXs>mHrf9d}snv(j0=^c9|%*AqNFZ{X(9OXaW6b z9k(PNPCaUs(5ZqHGP)93w*4|iB&uz@0o2c+&v?IIxZoE4@vz&>`Bn7lP^v0#xQeVo z^C-S?V@LMdMrfV!tLP$gpS%D_Wpth!1u|A@sUc%9^QlsS7>VXiD;sc73CwkO^@b2| zJ;Wmc&HXPGG^s>;Co;HdDQlfWF?*s_{?TYv$gOr>bFF_e^;Uv}GP`7{Q$>O;;GXWw zbUUsCzKpq1JRa^)V|sv(-sqKS_Myi~FCog6f1nS=ZOxdrm5&oJb}{$Rt?&#ltm0+D zNyqOi&45bf>xQ%+S+EsQJJ_lMmP=|xKio$V0iBK%Sjq|j{b!m>eUcpVWMVQsdMzE6 z;iEdakf>4kre3H5{}nHxb|FMAbVfEtSi7i2)vUo>m)nNY5;nGmil2u)ewr11xx#Xf zj!uV536%J|XQ!wC3_2di?gZ5obBdATn|JGL#hWK8brnOwlugyTy40KU48^lO#)`}< z=2G0yn;>niEkaI=XV@U|Lu(rb{@-8nKDuDUNtdzANZZdvfXofCSV8UH@$C_>fTOxH(=edH!!`l;z_pCOSGTVL>MWzts8hRsG34h>!U zyz_c?RG((Z?!i|l0akVqhnMRx2x-qdqxN#a=}|eR_Ugk&o%_A)PXOXZ$(c_50-aw9 zn#KedK(0;m7G1LXz^kndiNPa*KIBzrp>^LJAu=bSRr$)Q!o;*-(bG8#gT=qeD5(R$ zWaG>I$_a5GO)sO3UG+==c(_!W^hVMTH5*DpOWXQ^MbLbAwR=TfRjIHX)fr*3W3)J! z?{X)!^gx1IFd>&)3hpFaj_}F_fvKj7b!d}6=coZe5-5$R&=JSW5La>3S;%iX!e}}? zLrRb&_Db*{<$h&F|3$1HiJh(gT?$MJF^EL(BAW(nK$y>NlKyYyb^5HIcf6=(g_`cp z^X5buO3-pB;>(!?sG#}*jlS2OOX@oGOz@i|qi9zeu;VWB^&lvP^mHz z#j@zXF3P!3lxv$SR}vF5N<+eH&uQG2U)qTmreeIaIkF{s=gsTak+U%~Ke|HMNE_!_7qq-Xn_YGbP`f-1L!Cn_t|2vkrjIE%^iI~qWGF3RHwJ*VENxd>^Jf9x7+H4sy`Xu#m=4oZrBpF4;O})bi!iqHRKQa`&mvQHe zZ3-%;r*6Ah0^R(bk*VT|;x5D)%Al?&R`o#65!AF54qKpjHOxUcu-skF)l7j8J zb0(=7@!7kDz>A``ia|K0F=}w>FO}tnTMQMBx<+~C%)L5rZ32)eYp2vAq9C_Wf&Fo4 zXlQSNaY-{;)+&T8&v@F6BZ(t(aiA+{W|V1K3M3XDSSVw1A^x^9SFHb%TYhSc+@iP6 zP0G3nl5h(`Gug$^`KpKtH1vAWRDV64VOEu|;?Whh{hh8EvKw+4v*+3)IfNft=s-F+ z)B82fZ;*k?sgI)|*O6Wiq$IeWRzXSrxKTdA^J&dbn(*MzbyrybeDnx;&3^G$o(~HB zN!MP4Hg<+x!)crzoJ|ONdG2B~sx#AAqZLZWpw4MVnc!JnPaN}b^)mYC>(6Y>rgIz& ziV;=mhhDC4qzFdGssi~0o~5_Bn7`FD+iV+TH{sSmELCmU2L}phT8#2$Jfhtuw(QuA zWYI4zI=qV_N+{tp{}hj}C6wA8dk;~*$tK-;xHc?>ChpknE>m_2zr{~(4)^Q4lVc59 zB9cCAMu@zjoc3};Fz_fweI&Lrb*%`iObm0KLEa0e@@VP(1nE-C(vS={XVK-^3`)`W zOpKp(iO|e(JUZmZgyJxMc;mj7_NPj%PEZ4Q&{Iv?f+IMXB|tS{!YU;m|2jQ}9_>Qdi7N7H@kePs0twd< z?{=M}o%pT)l%jHPtvZfLwj993QazHtV-jLkd@!FC;(ku$rLvdD%Ba#jEVnmHiXXRd z2k>wWR0KQo4N_WW_6sGgwOSX4Yt(lIhoq|yXK+mi6aCvFAF8)Y$wGZX_1TRJOn`89j3Bpp#i@l0?# z9RZy=-oTN&5y-zbJe~)NF9PI?fMjGA6d>DcuUk zbgC7$vfM}N$c!aPTCbax5fAq^>jH23=-pPQeUFAlI&0C$-+^p0dFd#%E~47k%($&w zJoKqemQ^}gNVq3B(6K#Ks^W=Kk8g6M(aPUpV5xbaHdCGVol2-^j{wc6SjD7$`0A1p zJD}+e$1x-0@D#&)I#uDrv-+O93Ex6z4pt`{jrNnfOcMS)UBG_DBrg$JqpYtC3s<-R z8YZ$+7Ek^yK#Zh99TbF{2f9&2e6+-|RqoC~TDQ%2e2vXEXP=;9Xet+@CqsgIb6(p! z_RUUfgWZ8s?PcuAyWt$hUrUMDEE~@HG)3r>F72iB@vI2{W!-8F6-{I2@D;(_a?DMh$Zald2v`M63SIyIdD+T`Vglv2GMu2q>K z3!T`Ph>s^yC}2Xr_-QOXZd%Hjbe#e^W4h@xKh-XPlvvI&>s8(}@6U#ot?PHTnwaALv4TKDlh>ZRW$h|hODgZD zMUdbsN!Oa1z?AWdjDV6LbrVSIdv_B~q)5p0mE$mpQ2fQplI=x4%fOQu#>+=Q=WIf)98QW z^Z|u|7b(l$g(Q9ZR~mbwkt0h~HjXjht?9}8Jg?E0@JeOKx&uqGjF{V2VFJG)>fk*3PP3n>oorN8L25F{`T4`-|Ht_M z^DplTtTF+^jZcRNHni#B8rfqnObWon*k*!?6Ex#1*0!zS9hnpiHNGs-XSZ6Wiqe*<04TR>XB4CFUD{ zVxMDI-ZSZ{x!TV&HUF&4aH|JK`|eWcP!Y-m_2d5^pcv6;<^BB|k}`Zuuio-8vQ(`^ z$8yKqGi!j<&epAa98+_U4Mq!gi^Z;&toDb%_pQW@#R zfM;ap26GrwB*9R}zTrl+%tbUxgWgJ14~zZK(F+@^MXaH+O7=TzWGQCh4Aa0(lI1Ty zoQ)@<*jam%xwkKBRkq_uv@sXj_t3&=gI|meKW@Lsom#i@nQ!IF{xf#F9u3&z`5yVp zoQ=6wBafHVaR%!_nU&d&McW?#lBU`tRE=s88X2-|By>cf^vS@OPUYiAOS{3*u?p+J zh2av;;eJi#QdCQHhTvFJI6OleZ}m9v&voM`Cf0*2dC%SK%x<2v5cZ_unPcI-4}Z)5 z6?-fYcnx=19}5!S6^vgRtvNW%v|onExk%%Y_(m{kb7pGBKcW(CXZA4qex{GDefgO& zqGaKL*>nHL8AjthkFZ;2n7o=h89am3#pT%wWI?+chNn-**B0y^8EuWmb=NGDoA&H_ zffOBjz^>8CxB_w5udw`(rB=W-=5}Nm#cdgnzFpOmZO0O^n-2M)R~gBe)Nm<+>FAK@ zC}Xduz!*(X2eCya!EXJnTgY?ILglK-Iq2is)$Ft#Z-zL&T8Fr__aY|?7?To65+O`- zDw0A~?am$m=GM_rV~TJ?ZP2x9&(8zqLh>TQR8q*}Gl=G@yj%sfg&?7GgCRU+8KH@; zJ+1So`gE&STNyGZwZ^jbS)-Bq!oj%mti>5KG6!vh8)zDe(oA(Ks_NgE7ur9Ksn*2Q zUeOzysWnXx)2+yfGuqgBwk^&W+J#6Z z=-iD*=hs0x;#Hgb8f;N2(W>?vH*6OM6wOPV7KfN!k$P1JyL%VcQ?6xp6gM?dvH=IY zISo3206Eo{0KKVK9Y?n{qzP(nT{Z1kkhK&dPY9cWK#&eck4$zZ48A2Ae;q(#($%Kv zIqs;GjHN%lHb_mw4nm;p7d~4L?Z5rtlOx*(hT()moQ$M^qX_>AA>x*$xu7QUw-f3erTp)myTNu@E z{wxCnfFVt?ncZivvENmpm^v>ETrC^ljluAfBJkpdn>5in zotWe{t&zbYo?-jB&xM0!uH54VvmF}qT){!)pERjR(CaQ{8IVt?#g*VksU0uP#LTwz zNZJbI$rje%kGS__&r~J}@*w(U+x-s2%`N+8m~aUqlMPg zL%>cXFEEi01|+~p_11ND5_sn?zUf~l?g|+w0AZSEP)Ub+RpZBbFiYESBQNR4Q#}4B zBV6<1oMsq^28H8CFt(@^13s$B+lhos&rndmh_A<`^+nTHMK;TB3y#cZ;v)uRYgX zbFEb>dits|TO>~F=tJ7=(GAi(P@kgYJP?Hcp(l9vp|Z z0|vQ5VLXmMxGm>&N50b+!?O4KzQ}^u7yWT`Eo#a>Q_8i&h_r@Q*5CUF?2gZ{WIe}&HrMoAQW2ek``t+5JYD29$J;? zTkO=;ZpCO>y3nW;lgsfhS^gYmVq{Ava-|tMI!68~sWAjZy8bk$g*MQ`l#U9Jb;(Bwh_89(&QgiH5VSZ6J~Fp_a>Q;pGRgv1f0h;t95;0fsI( zVX_TJgw_*g7N>al%|Ao;2)dRqJq_Jubars^%4YR4Q`KIQ?$kvy3a7W6_ViKbw@tx? zl|!ag!4_-hvrDpGnJkYSB#Uy)7hM09;|Ij03+u>It~C#| zrgo;0__SWT7;ErMD_8~wg7l9ntPSRmI!P;MO6I6bW0zx*$dSF5d9T125S3xaN68BiP}j}Onbur7KGeUVL7WaR(>)Bi64Q)TYje^#jnsKH5ZKO~py zZuW1getnM_)qB*mDaGzAUnRl8@zS$dCv&VT2g$0ePtnR^8J+%~l@uyKU&(ljogd?WW$gLQlsCAMXdDt6#KpYd{@XAjT| z$?hjGY>E0UKk)aQMDKYO3$1~I-gvapPT)>y(`Ng$Zo*KaT&mU%Q9Bt~zNB!Tw1;i+ z0eZKSIIbjzUMV?+YI|;?V;=oce?azMPTAu8n;~x2gFfrf345gnrZ6b#avC0V6#am| zj-BO&t-J^S4-)%_5`W;N0`HjtijJudU;hQte|f@hrJktqaclbZPLiag-LQictx^c! zoS6FSc43#4@)vPj!6Yh%x7McfK8s2@GPhIg~OmQD5Cx z!?o|azc(Q9l*9SRr<^$1R`wIDSv?&3qxJy%M=*oF>V*BYgil0~b@2$j>=s!ADxZs@ z-5YofG;*LK2Vy6Io!V4fXA=50Ym_T&pY4?A1G=u}4gKNwpc3@#d;f3Y^Cy6?78&*7 z;{5&_3c3GfJ}Z70$R!}W z4h{a_<#e1tH?y(IP2~rO7W)-XtW%@Cko!w7^S9rnQlyr)V$Xb(nrM1<~Izs*$@ zU2#iCWqL@hE6<)ABsYv!RZor1V||}+pr0#e(+hQ+yX|kmv241^kv0S>Kzw+jBF2U! z&_m{Q{BpWS64{gvmM}b#tlt@C_i^7#Qvx`Ce0wjIb!6r~fPs?vRfdO-E@#{F-I8N` zlMwa#4e1z1^)ZMlupFqG)6j@wdVYPqX`;I;J(vm5ZwuRhWyU^c62|MY!u`KZU1;HZ z9h6Tb&cki?zkWyrdNEAuKQE|_d{1rMTT_ZIpy*^bmT5K6GLzaHmPg;c)DffcF3j!j zMmci;;Xg{!5NoRbSs(`{{1KC|H`&+>#Y1a20jEH%{k}Vv!qE2hzQe3J*Eg%^k-}lZ zea;=A-iK5ojb-C)$;?nif4pWTGEAe9@*@(JUB*nKskrymcYsv1 z+x5OQnjYob*P%FCROj{^l&d|85eP|RUDu9lKNedL6T0K5fMU1~e;31b>{%jxu7Bgl zlB?l;Wm2Wy)TyXTCWhG7nLlYUP&jGPS+3|$tI=dZvN?3=p;~5>;C6Wm8r#;~)ysib z%oaK4mCF(~pKNq5q}E{8uCjo1j@SZe4ggN0mHYOQLbZGiDdBDpx88kca2LwhLEYAL z+GRZu^z;W{8znxMim#{2Ww;4Vezh*{O<_|H!{^iOcG&r;IaOsrrn7kYak<0(XzJq2 z=9bZDX7SAE6+U)`+0JDyRq|rr^H%LIfO3@4=tt|XJcNYuKt_k_9&EANTUKsv{>Z4P z-X{!$?e}4WvEb#i{V4q&&jUJC25Gx6qlttsEY6S3d!d5-uUpX)jlmSUOjV%lq;SRbv(pZxoZZ(ZWTqTyip zMP8F@y0D-Gy?x6mD*sOK9n^C|X`#0=F0$Ey|0DUafQUhHC|**#m>p8I}rfI?t072?|LSJda7w-W!2W|kJNj0 z^a)1v!J&l?wZ~RFU$I)>d6P>MZMMDqL7nZ47z%~(pTleA{fD&^-zoWGKc@nUa3%qc zqPG4f$6e;e&(T&1EaI%B{hIImUL??4^a|G0))X9b$<)fRgkW)w(U0udZmmp`tLJh% z?97B|bbDK$1RP$0&m81gHKMDh8*l0fT_r=2Uz^ZpA9gt)is=|@Jr zA-C@S>1r`*enfOT$Kbkfsx8;}=x$IsNcZkL4~8X$_?nrO)MiL6Q{0CmA8FZaD~8#D z)?xW)=}492lnAi+;R7j5;X+(yum2fn;KVCV@)HC&`aquBHS=zqB(5$$^+xd3cEq7y zzy14&*nvytw%2ejpb@lef-*R7e7{P^Ok>NZ8A7G}+3Ms1d3BZJB~rAb@xI4ws3zcz ze=NeGT`$9h(<7~|^3L@fIu`+6Y7>XslhV%h4WVch zxCa}90^ZNs43{SYvnH~@>`hl!w!JBpBkaO};LyIzIBbdL6*_w8A*#soMwzFz#B7>V zj~Lt(V@^-LE20v9T*FB+WA;Fv;2hhD{WpDEf1u8LJgZ*hQTGe4fV`zs%;`dDu&O+j z?6BvxDDKtzqvYh|=;l|;PbRrk*^7L9@%GSJoZ;LC^JD>_C2gY zo3tZ^(?2(OlDMR`?6$8{@hP>EwQ03K^UQ_F1zx8nAg@h7WQYd6vRmGPu|_+Vs-FILAMT%&a`O_!LYFtJ#s$gBiN0cDzpIv(zrW zG*%oh(^Nac6DOAms)x6AS^5YEG=_gypkVyf=oTK&&v7u)nKNYX87X%UkNAWR^4V)Q zD^bxGg(2AG3mu``9mZ)KwkRld%H3g2Rhw~eg@@>vm~6cXwQv-Q7Hj>nWTYw_(O8lCf$-K7RPW>`ptiCT{yDoRI(4dHG z^&)3IHOZ(cM9~nEKpz;^pd7IRsu>-INi(&|+;HeGQLhfFzgWKZ-(D_=F(uqMBE6X< z*Dm>-xR1kbdlr3`O|4mgJ5>>mO>ohrfvz+L1@vSRrX$q_i5`BOGr)A}L+Ws29$^=N zg(WM}EPpmvJiYwx@>LkH8S${nzTumxF%Zc- z*p%9Ni4ce+eAd{lX{W=3|Me0SXx5Tdu%IT=D}p-Q9wpGx$+$4@p|h?VDV}}1PMpGI z>5^=li?B30F<4!Ab`Xt_>r^j9rPC?%nP8nv5+}S`-3-UCOd3+59|mV}_kAyju5o-R4KCT+|7@^t>2RWM7j;pUn@@-t?XWZmWsg!R@5R1@WCi9jT@qXR zclFTFx!hLa4ySc7_C>20t|1v9owwKre9_L;H6X9B${%9`wD>V+3Oy1%-CZiSO#_0H zI4lLDL+iy2mF^(#qmUaU0>qm~jAuzPe2KozyKEagK>Y;7EI-ep-Mi!ccfINd6wr!N zvX{}wlU zdQ)(Y3+WmKhJ`Juq<>&p?`JVYRgiO3*|&z4A{j);4E9&z@zzv!r8 zPNvojn2+iEo5}D=X8AQWwfR;9f#XTcrfWs97QyE4BOZ-&KBF!0RbS#%{}d^hR5SM8 zPKRa&V|)r44JUEW&yVQj&^wCN&XY(J<(s3#4=mHDb&sff^H6+a_qymN2QCZ1PjAQj z_cQIfzEP(A@$lX^3aS#G>g9>)QWBo zj%lkdRPLy?0XF7J`YQ22Rvgxwm+Rn+@1!YI29nz8V6pucz7NqB7?!dkBJ!5iBXzUP zGle=}H??m%E^}(LV;oY?lRvd|QP1t4nsgxIr=jf^|GR87{W-{1RR_!Hm`^!7Q~@Z&*iq&pU?`gYh(IKUr6wSv^Bt-P-!h198{{#eyV~Z+geGe%o_agx%k?pv%KRKc=^f$lS}bpDx55AkVx5dyeO-IynUEnWmW0MxYdR9 z_BJ2{n`x6fFLyk$&ut!wcVwx!f`A93TuV#qiA<@K#NMbi0aEAKZqxEH9j2*{+?s=k zbAUnLQQLD`pp0#R48Nh9{%CbiC8)ru5D|M$`H-);U8ofcIU<;XXj%>VP?lR836J*^ z!?NGj_>Vz?B_`4$((Hg{8dgB3o5l-;be?QrkQuXX=mWo-NV**b>5XMcI>HE3Diuvx z9+{mTW*6qiFzh8WA&JA)XFKpLYX*{qyf*&#I zZE5WKvbl{A;)N;p&xQwl0{N58kt;tl^;9XYAAr|@W2=6F#h%SZtN^F^Z1zh`3!^B==Ch8`YwC*>XShtxz&k81mv-zn2WV|#+<0ZTb7vRa*LFwVj2>~ z9%)E|{%4vM^rt{83UECip;S&Y3RR+v+o3t%Z6 z7pkJ3RO}x<6nl+VWFCNNp33d4*raq|JdC{Ud4&&j;*{st@j+CqcOB3sSF5Nwd&FbE z#ST7N>$FB+8cY3VuuEBCc8EB)&+?M0QOoZcDR+`DWl9n3`3JnQcUfVS^ z;qX`Ct5m(d`xDFj{1)49o@G#F*ddU*yRdy!otNHpp!98 z?F}LhSBU9)7|*GI`*X|$SJWzbVr>&rtM2Wj(lHzmq#Pxy?RkYqgRmMFjY>NG(aP!2 zx}oG<@5Fvf`|$xJ{W+`IH8KJY^WdBX622gbRmaC!kjh}>#=^EH68D@EeyZ(y=ZakM zg)pi4^1}B9cAFCdw=y>dH!eHhl>u-m%)+Rtk_nW!PpA2G^#b1hnNjK9`?mR2?e>6( zd5>xolvm0yx)d21jtqKZXqs$b9=X)daxc>S99lsE-#qi^%YjKXN_W_Q{V}m=vEtQ5 zwn<*F|GrN+NM~mwUX+bEI%r}uwY4?7+*uK9KlEX@k|EnH!^ZY*;SI`se4vhZXS%?8Jbg#5skuj#Npq`6jWfg#UK=dFs$LwrrD(c%mV%`-Zp4hN zWTEy9mfBf_BIr%alhx?=|M>fa9LNiQfdlO+X}6$|gl|c=6omc0<+bb0DO>}w$K@Ic zSah=9hxUAKAb#0CCFv$4MQOfq;cZrgQwqA$&z#jmKap=jI7AWjB3nT9{#;lhyc9(% z!~0&QH+I7BGhoL#D2g)8&zmc&LgnQ)+AF2t3**`MzE5&gj&l2#?Z0hVaj9`#(51Fy zb2V$LGVUoOg`l*YUiFz;`+-CkKZRg3F0B>g7_9G<5j`%R#M~dQ5c^nVXI}oi!c6 z9H)>l+_i8jW$!Y(N&eGpXC`LLDY;b_RexAY!T7A!C}vQ8vYyS1U;(H`^|xh8n2!>1 zPz|UX8IaY{V#>}YzMiV(-cV0gzmW*4HZYSNnZhlejJO@^2RiuiYYoWxWUVAufC7M$ zlB+|j!DomJXVq;#t&XND3;niiFQY>()6QWtP1jQ?MJy`1x zkiS;Mm9t{+v1Obm&RD)Fy;*yXDct^?S#e82h0~S@@Q6R-d75|Du_P&MfT}B1C^Om^ z3(2x9^FTeMGbPcsnEjgr6PC0V7FPD}G3z-7aSi*DSq6J!P~BOfHMR|x<5YxgQ&4-v zcp%)hV*~y4Whx75WM(<{h?V*EYaMTnAQ<2-UcSsYZJ0|@^ia0UwzdAwMu}NO3UwPJ z5r#?n5PtkR7p;KNI{$T>1IZj-0Y5~UEyv(t5qAX&tEUNtg3vBp$J1(DPANGZHvDdH z{B#DFIE;Wi%WeuZlhaMdOK)}EyDS`m%{NnN`t@Y}l~p*ZnYhjNQqeuJ>F;feQGIA+ z5tr1+y7iZ{ffa-D_U%z7tqL<06bZ*4WA|%06`?w%0N%F-7CNc8IBKW|3VJd?qaoZ0T2 z{0#s-1Vg`{PrUuql7|p6;rqY4Awp}^bo#`a@$ho8|Ce!p@Tad{k9=iu2+Mv4O%ZR0 zNFvCTwVYg5BwW}!xPBu`e|_312iSPsaxRt|YUII&&XK?0-lIJZ?!htvWkEitPtDk< zEMJgd!vqg?Ntb`B)|o+7S=`~R_9(S+&Fy}nGPJ|Y z2&SRS<|%q}GnYRHt8s$RD8i!1!u8s__1Qg^_{>2^Nu!Ii-LRA2^gHJoJoP5yf~ot< zfYMKS^7V~}Q*zkM7w$7)`^PzA(Xz+eAv3i{3kwkY>K?b4fy2-kFfu*YJDzHF-thYuRnkP6H_`I)}uyqzQEn* z;4mRu(3WgnBPx_AAdvWlv4pV{OQdk)00mS!!&0Wdh&$7V;WKEZIXE~d>COt))YN^L zzn%R7D&f&zk{HhDsAIv|oE_2|4LXIpaH14LqkH1$h_sC5y2kFj^nxbHV)hE>Xq_s6 zf#h)oUKyY620Q5$M?T~lmBn;Ks0oReS*x{z_9QqLI!pO;-%2L zwY^Q|@qFdM1Ic*5T>XXE_c1$buW7mqA1Pdc0v}|{7#z>JC z=DiCt9zHS$)bb$^`O?xpSf`KEZil%wYHoCr+LuB`S7+}bjpV*R%Xq!PenC`BaS(YrD42xs7S^~_?fF{9mn z{!BAszL4;YU?P}KiF%0DC4c|>Y(0_M#$2T<8BoSF#Jjt7;=nz6KQ-WFzF1wq2Hhj_ zcQXpk1Iq#(TsQ4v2I0+z#&bJrC zj(*}Fdd9RF(&uG%iNN*{41Ex^$NUi=O^88Hq9%c1f8>*lMaR)(Op?kad(w+Y-Q_G8 zJIr29`p1>gM%21>R&#)O@koXylOH*5v?VWK0GMU%HQxjo%EO(X}XT=9=-fLa-o(-nOV_Ms6$)g!Q`HK+letr6Zz|`E${(N zU1CPx@Tzl<_nO+bW6qm~%OJDEBJE~Dc4nKN7f-V?YsuZS!eHy=L=*(mbY0)>1B-E( zMat6k|0ISFTy=zTsx9XIyixF$sUp)rdELI0LSWW*ZM*9^P}U z?ZNBxCKsJkw|sZ_ZQv(q#8%z7Hx!r3^K$qtgM)+Yj*CTyj6f+(naq8>m6Kj`vvLLJ z`h@DWORaq=H8*@N6X>}j1)=9zLw6M@0T9HEe1F~+61%+#fmJ7ak=858iXUL%z>evGoL8SyybEjKL1&8;m9(l195R z*oTzlIh!XWS;l2#Wf@JU{BHH}u2 z#9?`So;A~TT#M3w)2YIE{sofaj|Cvs8d{aCsA*h!82f6}G%K{abdO{)AX6b==HP&y zCP-{KlhpRKClMJST1mw~0*m}YWfpw~voruG&(kbSg;%YYCI}{{Tn*|b+g@48+b)|I zY>WBY{Eaq4$1xNnbWpXzW44n{3G^f%^Y=UGL`0NsgtY&L=s4bgDcC|jf72a(<9_H|F%UNoW<`LcBJI3JH8#E3J$ zaa^xGX2?v=e`m@n0Z}QKW)yigI}OPEc8hD|eM@;<&zOIqjqxIHE=T?dHfi2_uHAp6 zFOdZntX>%$1*^z3%&?KQW{4j&I5T`4r@J_t7i;#AY)qv`PK)8bD65JQ4`S_TkM?HgZxm%D4G?Sl9X%%Xs?5y*~{BrDE}9SY1yc$ z!|f>Kyrbln_3iM&I0i`WT-Y@z1_BRydW{30DbY&d8breJWLpVLQ2YBSmg6I@^BMiE zZ(9!!bR?SCQ}EdYMMRdeyJyva2!2hzOkE^W+#le2;qouba%${TJ0Bt*0O_ld&=Ly1 z%<3%FeF}c_n;GIJf$BovYUB>qq*Ff%<~3JC*;3?B7HbrrT)+x~^J2>FZ!ex^WB7|> zY^lu^T}*GuTjK{RER4z+JfT(itn~B+_8ca1_ptO82M34V^x3?99Mp~9CO8wtgjK?s zS6Fz?Ut=6(Zg`4KvZq$AQ{$I2aIt1&a+lbBs9G)c_{9d7gtJ^-x_~RPCBfoA%Cx=8 zw1c_3Y^jvM$L#lr@`C(YJ?zh z_=}3oyyJ3alZh8VGeS;<-8~*pqRJ&A{B!`R`zX&{v%9WE=S-JPRRP2zUd#E7)?=dc z>#nv0ED-;2hjK}wRg(}r=5R6{1yZkt=DYwc=}3q2J)t=Ktx*mUf6kwa(#%4=9Ta~= z9&6RUL`p^E%a#W&BK8oTX)2vy8Io|DvL9PsG4CO`8hz=Z2F@)nPuS zvwEvT?rYwS&j>i4f~`*_=?%wFR%2d;He*NGr|AlbXL?H;0?npg5es%0wi+oYN~{yi z;|guC0HmH8dP${q*4mYpU@ydCmrZSFrl4)w4so(!SB~0YsMu7; zcW+ngl?}cN-AmmIhYk2H5f|-e&4uGdBf3Y}q;?k5jZgNSszqGwrHOB8?@aPj&#LE( zeSA95E#-fs^%fm+uy_%6{iKog?%2d&6Us zgBZdTK1Hym?RtN(zBaXB(=88!dAFGaVtj_L6gJVh%kdckkJIv6+F-@uw94(JV@)Qv z4}v}ad&6R&{PZHp?C*WAZPg%1rDa9+yv zOPjOeah!AGllvo{qW-dkb1VlS>tu+&rLKGpv`_SRcxiQRCGvBXkW2WhsV-@ArBo4> z;XJsu|Ljn#x1n|{(D`&HiEDg{S8FH53VLaiA~mfF?Xc#=>ilOdi6mrNuhlNO*k1gs z{9|{}Hd|>s15xT%Y^IMRM4pn;Q+fg&s^xTZAmf-HW}8#F$P%i8DB*2$tHz zrS6hbYk$sxo{Pm%2OqBw=#Flovs+n}?kVTmIfLg-A8BZ6T61im0QhO~Y&XeaGNM06 zJl0#fUTUJs{kg}~*D?0FF^;vq6eK+N`V7%8OYF`^>`Fx%e#>OH!w!^9s(5q7ZV@!< z4m-0DwpjPtA4|Ag4jW;dmD@O_G%@eI=R5n<*<=a?v00k>f22?$A>vQ@lZs<505O9) zOB_pWX7d@`FewV_jh`M8sauu?S+d@_Mq@Rk9Tcpw7Z}tsSj)oFZTOC6$GCHJ9S%>O#I$NBzfeabU*>j+EQwPIY zuk}$Iwi?=8-6@+wsS=K!@YCjMajJkDWP#2_z!Tr%y%U_BLWtmXf9l5U;d998CkAA7 zFK#Ubd4}V%G7~XUij8-Mz`u2Js`;WAOFc>d(McOD+lazR@lL5`zQ!JPv|y~Q0YU(9 zZuxnNS+rN@c<^MPcp6DS7a8IpN#(fjw*bT&x^JT@8F>ac|6EvLO6VC&;Akwc)IMt1 z6Gs&u&X)2|<++xrk+Hr{hq3LTV^h&0b4ZsSibW^ySV0Y^ElpCRgDEfdY59Klu&& zUfJ8F#r8FfR%V6ZuJ>vAiJ=!3*vcbgV+Y!AkCR*YSH_zejI6wMnr-q#k{v{{GX(-& zm5wG}(7qMt+hE_crN!2G+c&K$e?~{VFr|KJ0<|yVLNEga7692#0+W7wOKn;!ha#m$ zjTQd8cX6kJC-Mh+WQ(aT@E!w=nt26jpVNv{KkvUZ%ryJmSee3%VUqh38|#9Ji%$b* zJx;X96VqmZFi(G(6B%#uN>>bK?XgoneSMVOILGm0B07P?dlg~yy6%6ceN&T)HsL!c& z{9B{P^L&sHp9Si~Fz8{-)L3gyM^yv#8D;6z)|FGI_B0_KveGiuQoeeL*^`=ltG4jy zN1TRdy4jLZh;g*3A^?n)0UKFo9lT1zO9i@NJscg;)W5^(sGN*|0yA*G={cPc)-J%x z$rZ?a;Kftl0<|N=16czFsh=I&z9K(9A#em_*vg(e#)yE2%v)*xsqLYB&kKa|RT&N4 z`x&;LIk88>6$v#pT9f%DsTAfjI;vXH_1+}>^<-mjCl!a2-2xtNeeVQTb2UM}Gn;6S z@8QQ$x%Tf3IeUxnr~M|ooDj_t^gi7&N7E`Qsb88MYA}_KqHDXA^6(2S;1=09i%r&h z#=$?Ekr`i=M=A&>2Eq!LHnZ8 zR5<8Q(*7y*vq`QuA43*h%J^yl#*yjF@pI>ZaP#7%-!yBTkBiOq*60TqH|8huZ~FKb zKMS3{MUs{#{lv@6d+4#SfIpkcWr#ohq8#%#+XL2xUJ{~cS(oPPbdpMx0tp@tQ6jCq z+?i@X1p;s%T)KhW>VEX{Tsj%89LGfhda*QAK%~}5)>Hzrkt&|fQh-wxZ;wO}ztOFp zj{c(GwmTpx3CyX80|4kT~sG#zezZyu(+F7#{`6&J8i5)@BhvPH&KcpbxgGtc6p z`?PQ-MM_HQM>IqEp7NgcUO%9_Z7 zdvm^36Yj4Ez>&ktILZ<8DZ7Z;Sy}u{)$)&{?yEHSimreVdVKYQqAQ2mGX-?ZQKD8z zqCJQ85mwuC#iKB>FIG)9bUGQ1eOHJ7;r7GGIR z^a8Q&pwid^vR09dPFIFY0d??hC@dxOedwMd+eRuMA}p-p0k4(z$4}HoOjyhR9BwKfe<38_)_VI)@&y5lC=Xh^aFbv}PkvhsMjseK zlYz>LXl91d2$_!G_h(F_lvMvjt{^|f<12(=XCcxNv-qPQAtB(>cAm7Vlu^SF%zPG{ zj$oC%Ha3x3I`11(%1*;sP6Yb&efnL}sSJ0_6x5S{NRF7A$(IR?AC-t~3Jd*-N=>Yl zTZV%Wkj*Wr5CH*I``7P!c0XpJx`UNk_JS~X?>wf{+ux_L;t{#}3nL_9H*h-cUp`+_ zo$8pzd_Z#e)|V9<_njBoi6r>@%;2$qs7dV{QXwdB9lJVNM5KC+aYbeRy>!`HtOjWy3uQ3Xny!n^QAR}_{RMLt$f>_?Tgnm8qA0_`EjgsEI zM#*UTca73_AbcOKY{Cp}C^6<9giaHP$eUulH)Blmkj`)iNR|bSf8eeu;tl;?DG! zedRv(Mr+;#HoJ7Kr9O7toB1~P-H!>CH#PS-$b;Df2FQJf+gIBCnPmu@Ei;#y)^~>j zLE350Y`0zDvtX11z4xx#O`84?qltn8<%r}6qlqYHHoWGx`?}e_3RQ)8ufI~NE!c2< zq;|{!T0DEUwx7;%v29o$AFCoKpfkt69+p4&JO2lwA`@wE{)(&AVMoFO2oIY`>8q;s&P$&oo_=qY*ohF!<01rk|^7Kh3 z*OBuM+j7Gr={Gpg*N)WA)GXs1&PC-jq~nREDQjObZ^sK8^yA zV~s?mmi06cNz-yql+H1=@b#aK9~P+v`9eB;4!xY}RZXw=#}VKfUuECDBw8%*j-~~r zea1ZBsy*g)M^w&!_8UlWa(^~K8Ptszpsa5ut1z`&Ms%}F6T731`lel-sfwI(yu8!z zq}qUwn|ox71KZH@%cUGzW!x*sLmUj)?$=L$iH+6(u@S%{R9cD*nR_l~*gcu8eQsEZ zX*5>EVYQohiP~zrCfNyoQVt|dg^8D?ewMDAa5ntIEd*Yky^H{VMG3nE@(Ex(I0xC8 zqgTvj!Ycu38A(#e# z(7}L>B^*R{0)(hfDa(Xp1t0>el47zWZ~vRDBp1d&N;`e{D-a)r1-i5{^Y~Ik8MaVa zdhE=evklePe)rrVi_pa_OHg$Esi zv=^xNMmiN-XMUdAbh2KCdbhZkB?7pwblluLF^{eQnWfE9^YUS>?b@LGrSPA;&}-mJ ze(Et_7IW;!|M_5%Fi>o9FRpk?W@}8qX|RKXqA8pR$#R^;VxtJ87hC9>lIxNEKYs(5 zf8kfJkHP@!!l-UFNBbwpc#RJIba9w8rSLEo7Z>*>I=U|!#Ul|#w*SdEAs+hwem$Tv z&QP@+fUa@5JYWD@Xi;uFj7n^ChqZlWGSFT@^9mJ>CR9jQ~J!USutn78- za#2G0o5Wh1%x7ePG>g5yjWyCVWkQL1@px5@ELl_c)g|?|8yoYB3?ibyO$QETK(jEO z43ItJZeHv>zpcPuki7?{rDbpF_QuudLP>9DrWOIHeL92(L=&OOL`3X%ej)%>Z(0E> z_A}i>&&SuFnDMAlNbd-i?RKz_I8ksI%Z`KmUeYGX4s19rlcZH&G`8ESbf2ZsjdL#U zpbU$a*y^g$^vA$NvU-tO`$nF*AMO3I4+1YyIJ^K6_`+b8h=C)6s|6wB#~(hEOqdyF z2}cuOH(@pEF+uSB(I9jY})rX}2|6Diq{<_@bDUOoA&%}fH zPvy2Icm|4KM1&mS%Bde|r7s!tucLk8F<6-S;-t%xXp&sPRT@LJnF@i77!Ys(hbmZx z*c5GXSOJYiLVJol=#|ky)bCSxK_6wo<5+*FYFF(9t1=YT^&t`VS`QkKM=6%HlmvJj zJn~|CV}e3}h#R=nF4mFp&2AFfZ1$F`JFYmu5j;(z+&{za2UIwRJ+R^73C+}Qa zdA|91h_q+v`m$W5(&Y&v?o8osEQKci{`sp%%F4#n%^QD~w$TP+RDq6E?sZ)ACZ3<7SwlISF3DPuN>5qX0BHpc$InS+b)>vk%L#yc$ zSuGGnK+)`WMtpCiPua=`^WMobn*!uE8jiSIAE!Edo-&B&>J+;OrqaL0(eiWS?YH*% z=B}=1O_q}{vPHdsbh1qQX+I2MvLcFRPxvrAzM!&Of#O$r@`b!EjS#5BsytMXbX-4A z8YFp1!w2Dy4S>$~SCrJI=~B*PBm^^q-bC>G@3Mh5QhM7f{*`0Obm?S1$Bv1`uP9ss z0s^6(wYICm6r>hf4K70KjJRwD#I)oY_c|&W^*{~$z0?i9ytuNa3`YASGj6%N=|*=d zfYw9E5(!Zz)*xZ$;#wxrC%M*QKii!TrHnEC(A}2cG(Ip+KXPH*C+t#vqQbr}o-Mwf z2}$L)pJ5jUlbEM0uLLqxkb~{FE%QpaDLO-5e6oyFdIp!>w%x=I{E_p28d zM%Qr5_Ak)8ZZ`h^FkJ;tBp4!D?O$bSx>C|4x-PHvZEwsp6yuLI1gBL^c7XEh#&}-r z@Xg@c{+5aRYw59~0tf<7Tdb8n=i(&;$l5ei(Zq*~Of>NUpH=@|GRc#bD;2V-t5J*M zlhqkcPB-A?lnxemJ>Rz2k#Eu>(VLZ_P|k(ch*n|$sG^O_afNg{aYps#iNyF^EV8yVkWmddSn+GKZRf$UcI8;-SpH4|Kt7{uMW z31*Q)jL@KywG5?%_kYHuW(;Hhm9c_BgO^o}LZac1Q4%bjLaiXp8W-U!CrF z8#BG3fZ`Ag&I3uM$Y@1A)Jwfays>>79LZwZ=JdWwAyYpZMH=BBLhJAkLW_Cq9YABn zbeb8Wk;{m>g8S7k@{eA+f5M-2Apsq2G&u5U%tr;x+HZk#1`w1#?}sF$wS9v@qfmGO zl)iZfWYv`Et+To{?cwO9nJAiMSZ5iy^fOkw!#dy$1a_Kj%ITL~daljmJS!Z5G&KH5 z;i9`2L+y`yc>nxb-(g)mXAx{NYO(|fq%RbnwJa`5w>b%rA=kQ3JHGQM{XEryNrize zb@->1V)=^~&efLQW0N!n*Kqm{Vs^&HK19)4C|*aiV}zPjEe@JAJ6jApnnPFldTm^X z#-!3E;2$ZS;=Yq9+4cPi1aQvSK_K`z92>SaE2C?)#a?EG0<+J6#Lb7~4sUEOlDH&@ zoKmP|JBXfgoU!7)s~cC z=t@$(S3}9ArpVdkyVB+eN}uEIv0F0=x)$e(#XJpOEB&eCgh`VCYTZbcX(^v6_rXJy z*UCcSWqFps8Rj>GzA)cl zgLnK_jr$4;Fx7?{30t-NDDvf#%^hPm4bcPuj3_6XBb%&!7 zh6;;GjMZ+X=1)jzP8g0x^XST-=_tY%w|*qC5zKARcF`+%1{oeECj@^{D~s{icY=^Z zXyfZY_^I;!$?m@n0Zg9skZb&&YVAFVA~7*AQizZ4SpEO7_m)vnwqN+DVu7d#f{3Jq zlz@Orhje$RlG5F6(cLXAjdX+3Ees6JDBVcI0K>V*x4_rm|6S*-v(BgU!L@h>dFFZU z*!SMozV@|Ga}1kT&iUG&iI-P#1Y2DBT2h8V}!7p86^aE_$nQd%_ zV@7c=!waMhbriLuZycg078=)dpbF2L^tk`I7-P zw_HX`C0kp1{rk#$En?diz-}t_995q)lZ=O6L67*Ln&xblI6f3oqcE?S_X;h5fSD>v zp9({sr|uz>1vNsKlD}46@E%U#*i%s8KG27|Xan=8!89AUZ>E&z5h|k@iez=2pmHoY zWN$T5sP&-8TJGa1gb+1>5=Y$SDS?e%QkdX@G(w` z^Xf-5#}YZ5>uH%GD(A@7UWOzgA$7XQQ6@zeC|ZLwZ5G+fHIU+J4NN=dJMrGR%^-%! z-Jn^=bPE%64&)Q`Qt+lBanFEy_LSZ`~N=A-KeuE$o#wzjLw zghxyUW?>w*6N|o7O6ix^1t*$-0>I0S`pJXcioh{+o9_0_SCD!6l|8FK0Sh6C8V!#9#Vsk{7@-neTqHNi6x<2nR6)Lcbw# z(E}(OGw$bqXi-X49Ek}DA=J7Cgx2K~GZTpmy=|zjnOE~iAFt_Vn9j zyyme18r+-*EV*wFa5fnq?G&W=_arM9YI*sugg>ryQaI3~P7(XMKD2!ayImWVEFxu<|xZPJ+XGs=nKjz$zpXbz?#U!S554AOl zKjRU9%eWn)k)lo+S+t@r?%tmzsl`_(HX_$yclhIbcu@uE<=z-}ToAD||B!uCy6|C^ z2-Au_3u|8OpR`EIS3aFw%aa4blOF74+}WjvK)&TeXZ(;%`3`;@f3+=r zwI=6l!Sp(-?S~qglL`oERJA4)CIjEcLy@K+^7p)Nby~0LuDVn*7kp}j|79(rTIvDC z2Y4xSAHQ)?%h@yxGDns(`-fh!t0L0XBdJwpC&+UPVQN)>fBk91D z(TRgMs4h)>%DRiVOCP5Tos#l(C0*pv`xyrPj)&VY2ELugmAX5>2$&ny7-@e!#?yFO-&} z!+v@&PsLX_TY`Ngr(I+I^gfGm!G>US)dti(t)(|#tk!JgL-v*}Vfq(@-vwv<^QDmN z9`k1*MiC7FG2RXDqe`)fzs`RY{1a)N5uwEp%3HZ>l^VtCtv4&Rbit?hH~ z+vcewo#H_m5mZ`o9)6zcWyX>~wMC|98?zn`N1r6xzRR4o#(|KT-Vzc?8Ue+@EAF&P z?&br*XBo|47QKd(JmGFUdsALS4N&HY9@nDqLn_5dECzVBz~?dZe6vZ!sHVo)s;??8;X#x+2}S}Ox%AfT;e4v8ITRP~}6Mu&u+Wluf!GkDLq zH}W{((*h*y>v8q^u~v?jv|0eka+GwYw^? z&l`OM1Xu%6+-^b|H?IddQ0>QWPJb8WK>Yz@)yb@I@*lnm4>&Lis;lZeuw?L~=a*0bkjt=kvX1!b#h-!49N z#EO$@Bf`&|G0j&mwvdd$w&($&Z8M`;@J&3^9(PAas@dH~g7XKQ0bvaGR`|^o9H|M{Qncu%e$YoSe>xSaruKsVB=eM8uB!d?|IQ;SG{2_nlk2?#bWo0K|fT=0x{7y6Q0 za)xLvzkSo1ER-OMLi4h8PAu>hc6zkUwbu{Dm4-QkWQ0*Utp;7X@9jDe>gNt`4V@MpQ=sfOHl8X zG!@JB%Tm%(2kal<&Tm8m^%)o6*nNk`Jg%RFo8AWqu1`40l6X9Kdthj2==#Jy49*{n zzPCF5&QTn;p~b3wvI4%F_1`rF7X)6<;c=NEU6a#9GCbQWMW_V=})i&b{V zQN&{{6p--0m$(POs0j^?p+iC-YI0xb2BFD4<2^P2L`Bfa6LHv^3m0fOF*-Lpx@nyj z<@ZljuRvsS zQlsm1w_h+5UJbs-ehaw$|5daHD$xyZ&{CA%!sAq{X!wI(^=xNdGlAO;3lOXXU~PLQ z*(Uj#C;sm3On(>comcz-YrN4!`VSqrVT4KC?8AcrP9cf_1t6Aq8xzmy?Z^i$`F~0L zJ*j;2hKz{Y^~Gx2c~lQDvnZtr+XnQ}`^POjN3oxoAx2jn*R%Ys=EQCTvBcC`&TG?N zTl8y_g*Bg~K}s~hjz2>q_PV94&i9a)Za+EKy5og0*pD?`;+-8Gb6}A_>-A1*inaUP z!Lt5;T&{=7)O8+xO2cx3eI4$I>LCP$fG@@ZL{V?v1afg%3bQfi%k_}((*-R3OoPCC z3|+tSsQ~Bvja&*WH`Nckb#y4c8>h_@}L}^<4U}Zfj^`JcZr?T zH~*050P2CCdPZW;S8x{bH8yOb%=X6gb*gstELbGA#w5c^TnEza0ITnza8#~SL{^p~ z-BlV)na+emFS!2VTK1Q@_Y*f63GVak6ONeOH7*lL(GhcXK>}0n9g(apjN4+NKdMOs zI?HaD4G<*TEaf1EAN#pm>bWCB%(pm=d)<>KgzWH;QSXfOox~-2Y zt!ApN#teHC-un2-2~0%%RDbJe!eiCHWuASz`Fl#rJ;qfsiP*az1Q(a&7mTHWlD%~H z%Fze&$*YS~C3Vjz`OGKTExtp0)RCwZL8MPLQXB4t*-ve_l)y9Ea62(ekojzHPD0DQ z2_p(u=wqo|mG4*|RZ@yKEAy(`j1^5t8ClQq0exI>QzGwF7*LY9|OfKH`w;Y*glO@&-;NnonV^1q$H<~m+j5WX12(p|Y zk}Q#6REulm?-%rp=vr=(uG~NsD6RYDR?=iI3aM!JfDY3b=dtsyu!;(m`}mf40==CH z%h%tL#i$!-g@t*1Q?ZU&d5p#!-IasPng-on?f$9MGeDwLi(J$pE%}z>L0fqX}`CjfksK-&L9C8 zc7P^E`7jXk5GU@$#JF?cL>P0}nFS zbFyaT(`7u`L!xOg28o>shVBf#59?&hC=e3jM5{3QRhZ9_5X}cy-wKCjcYKI;Mnbyz z8RoS8+LSM-?DMgi`ugo3YT9no+W~rHXLEEWn79xu@04 zRjb&5xtAkT4g}O>T?~Wp*oe%=Sep28m{|424AbZg6_v+KYBPj($7rz40!XMImpT3KlapW8O|Uw+SeB$0G612(RPP z_}Q80@J)0gGVSx7$AsRFwfRudP>|1-BT5$}IsGZ}iawV+Cf}u1LwTy`u3puP9YY@}p!fM{66Gvi;LGt9eiERfSJoz`6^V3GNwMb7`A@ksXD2SS&;%Zji#R&Z`1ULI%9qFI zLZVYdL?&c)dESO&DY}gv&|sEu1BB$qA+En|^hDM6H=nKc2syy3k0jgsSfK74o@LuY zyZ80`rGxo#oK||}2o{U9t35qm=I%@9bdv4zU_R5eVO%5I^%lF)kiYKKS-ROOpCZ=% zbfP3nm_F@?@oM9FwE!*jrcm0eAvK?aE8xAZB zkQN{J+wZtfTu4l!Ru@N7@ZtOT?*;uAK0H5Zr74u5B30xj6cQ%F^ zBPObP7B#8T1W*%Fwv@%Ry1u7fT?)w^aMylkxV!VC8Gnuj3DRVsm7w5$>7l!H_101# zDh{9WmlC1ucAD|>(}I@fi|=8PiXTUdemn-QkA`wB$*^mK7_oW@CZQP@mbj~pMC?Gv ztG`MQBZ;3pHORIsB^>j2JOWM6?J3Bkpx<@g6l%0`&uyw^SlfZrha!4Hn8_Td&@2;i zQn&i3T1Q`;4Li-c93%|GxN3qwEf`NJtu<5_iABeub0Q5(1FlHO<8oYvq$W^-nDVNB zdhF6pM5o|#BnwYa?&I=3Y}nppf6?`CZ6e29(D*S-wSa!t^Ib`N+CK%2A;JGGKr)CC{dZ;`WD%VtxNFuxJmT=KYzDH6F zn*Vs)<)^F>tRc^F-o0%+9j(Fm@zJlqYsQ=WEiV=y-_4XauXi5$`T0DdP91QOKWx65 zN|H)NiYgLGBLUd+jajH&#y?etn zplLMrImyE1MT@bGk^vZYN<*v-k71Q+SlgX6YU++&pK!Zk|i+Y zG37kNzfvVV^ihoKda1tuvxkh%ijg^T^?ZUh?$90Oyy3OU3gN3fBih_Y$GYKz;wK(+ zda*zOxhfO>v@BgY*u5AusR{S7ReE101M22IK@HnII3hv$opBfQbTDKt`5vxDUZ<)>*G5+eiXr?4^W|XyQ$uZ7Skm#A3#UBHmQe6)`}m6wB3UF;?dNJN{Osb? z)F0X9OI=-&3`V#is+j@0d7!f5ufgC3kh@zeVK&f}`W?DWm4(u(a{ zXBGU9)p>VW2JD`{MLax_gvTcxe(J>~lxHf#UH@>5^(kWzTNU(#?TKxA{`>(JK0fWy zKIC&Bo-FJU1!(AXuq}cOQ`oIj!!$&;#xzE@#$0ik``IXy(pVsYD`mN|8+1SNE=f;s z5SI>DXD=^Z72iMYRHY_!i!G?FPN%j|l;e6tX)?=yMC7=cn64Rbj}77xxX zys)zf;iKs=l~fz!ZfP>4e4JAK7EC zgPQokaRWrFf#TmPVWpa?pGIl&*@nj&*ecEO!d>VELVQdK;ZA^7u9ra zCQ*rk1s*R&+t}fW`E6gN@$G;SzfXyLk|(=p-X)p@DSZCC#I4KSqSFaADW$_0HVT*E z@u>#bsSVjY#9kvX+nuWY10PvojB>65;mCFAeGIPhg;8P9YvRuylO!RR3Fru476v+N zcI|*>ubN7oMat#|ZL5*V+_Hw${K3${l@FP)V!8MA!?K6$9j_SrV-QZgQ!Cl8c8Oio zmBLf2Z@8c-xoH%pEWd)JiD`}!3@LitU1h70dpLZyH7D=X?IQ4egVykP9eCUhON*-) zOW_X78-f$we>Z3N>*@Y}PE_oRSZQ8ZqGMivH~sq`T{~}G5eZ;V|9<&wBZjlj@z>v9 z2JJNad40*(kc~Hf+i>%*UwEUC3lR=jqWs5yyNn_TT2g!v2($m)*PZ>$lOMdyEhE1cLUY#d0q@o=E$LB+K`$z%3LA{?O{PoEH>xH~{F-aKjQ-!7zp0R#sAMLNvA^9fb z+5D0JzSM7jd1mN&KcK$7EXM!p+L`fA!t$ZRp>%V;88Ry#tv42cHqRJ*(?t%5Gev$6 z^^fk3R^)M|y}A4}j6Tr?S-Yk;let?wTXx0z;K%11!_mI9v_Ksyn!Rkf1GK`>1Hx6s zFK5(jGywyC*6WBUw7p$~&C;?VfbcXTfzydoTz}DjWiW>Ur0*h!cw8xyUH9sD-IZ5V zQZy)*ATA#ia+UbYEnqZVQ4FE^*_tH82Y>&UgOtYZC5A3qhB|r{mMX;@g*+Xz&&UR$yA{d#9=CFu^?{I> zg7ta`HZ>NRZc-A-J2KLt!{qc$JC=@*ZONaeK!Z^I0n=>d3a{gxZ49|`(<^vf#tCeM z_wUp6dno}P=6-{Dvg$gz;_4e6VuX~E zF)zbXbCul4B55@#q+j;t>}6QV)(2R#<8 zcI?;K+L523D6oxHppAS*$mm+iHdXZ5Za|qr@6H`8y+0a+O3rP zrE47e$x-EfSexN2_JGx4bktb5&4(6KW1ro8TS*sABr)Of*~u>O_^SQwKA--Hl6ld8 zmb5!nkz1RRYVE9kmB$YX8!p8b}0){9@Cj~3gN+@E8UpwI5l*vF_Ecb4qB zYnFEul?b;hj6{umM8LT9aCX?$myauQg?E4a zM#xbi24u?8XA(8_0YYaDk(8WU>?g~dpigub>xr}LZXb)>$d@r=wU8ILKidkNq!8^Z zEWJV0gL&U$Su|k2o=sZdZ&Td<(rf(wttisaPpUM#dS9L&y7tscZ1npUcb9J&r)LqB zu}V^0+lHa7GIvt z0sRamr3-WKq`_6Abm5gHOudObl;K$!LFn)1Y}JuFjpky+nz+TLhdW z;2@Wm-*$i1yELmCc)x76*>^D8y({P0u##xsh}@@Qvm+xkjNlNIE7v<;M@PjPIpRKw zGtB(iIJCOQdF(s0VPB4($t5;7_8}Vf>KIV+JgBzjaXLxwysTBXG@FS?L-q~jdq=7_ zHp^zsC;8m<8KbOFpsUhDX+nk;Kb-LSit?!8D0(KePNGS8`{#qf>>$G)4U#K^K9iF! z_FavLQ_A6w)NffVrpTWg-`A{LdRrzEB35o6jd0s|3L(Iy#-fwgj-hj~H_w=M44n8{ z={uM^EOU!0t#ufdj#V8f8MZIuS{n?Dogu#yJajj0)y%o-z8pXZ-P{RmwJpzR-KSlOr$x$f(~7ms5xF@yeWN$}pL{Mrs`7kAyY7|f*VH>#jnx~LSw zg0Aa;b|w6-A8JJ%U5`7QX&}LNt2L56h?qlhd9c7x%t`;d1~cEF+-&ji6f+s0-D{hJ z8I0H_li|~ciPe(^Rn6pA?D#fEK9SKFIhH)BF8^e^J|)$gBpJcb0GCj!Fg77HW8dl+ z?5}cW|JjAKi`A;zFjwO*Er6}`%U3cUQs!TStS#<3*^7uU^F%bveLJr86ZwwAiQmC216M9&LD( z%1H_mWG#jqKB8s9$ZoH;)#kkuWU(xcucxz6PHXamq<&8 zUEM8=bh){hZ`)UOIS+r^g&$1-bUv6m}|hR-7$>1PY`5UvwaGA<|KJN$DRk-WIP zL)8fsp*qHCqYZCotEL4l3QSluoIXl++mwC39d!xwdnl6$-1TYBljp5L9-RcxG{HA- z-aLx_h#BzG;50R^erU>|@P^B92G6`PVV0DOR*q6Gy_@q+w9D07Q6LJLAb)D-6nv`z zMvj2*)eAoih^KeEYAbab^Q3L()d*u6rO2-F)47RvHlSZ2zG}_7!y!DrZ*KkKe)dy| zHqqgv3UR-=bXcMP2W#MjIp+!T@ z1i5`M0oS7*bEB6+!4D}ZyM|%zFV~OehKWIEF1p!P{J2h~$b&&L35dn>=bveu2`doh))kfyL}{}|zUUq9Y7EERqN`c#qA7X(b{cU|-X{th5^ z@k;YZZm*ef#HAm!%Ld2DLVe%7&ygJ_RIuh!k&{Cl4LrZ4;6-}Izx&|>4mWuG4vP&! z*uVKzZ@e7o-@A;L9(~cssstXW;U+C*3R;z!W=3gNbFSS_0tzhAR((=YiN3sCWP0;{SaMblV+72KJ&>d0T;vXH>OITy5a!t`;{%DVg z%KvU(bTc!WasEJqq#}({9%;sqT81VkA`4q-hc$%Y8 z<)@q7+iWwH@DmzNA%#xTkLiH?jBRu zOSW{OJuM69^D-ixwkDJozo^@ZPm(!oLdnS=wofZB#5+a>`OBky#a@V{6b*p$Dt6ScS$#n?z*#(Lf%I>GZ8pY}*5uv%NR#RH z0eL$ukI9=`Gkv*`0A7f1BcA+xoE=@VX@jG-ZWp=6i(1u=#o}A#GxV>Vv z(;}Z6_*M-XGV+Vd)YkMl)V9i}Kk`K%ax>|rKT=_BlX&Vfzlv8vl<8hTbmg$ug|^#8 zSf8A7oLDm_c_lCCaNAp@izgL&Flb_Rm+JPzl0nWQ)K8@#3L4Ud&X773kFkxfbRN$aqMT;_!>YYxz^Bl|YRA1@4no ze&+K?knw=oZ;+sPW|PoZKRb}!oR3=`9t_jsf_JP>a!8U6sNzBShJwsa5{EV^pE3=~ z)lYexzKI+tm%IjvLSN|Xs@Z7JhxTe@gjH0OnHLIGa~os5ryN4pK>pM-1u?;q~E(sJOp&(D?KcFDKU!WjGG|{>xYKz^$O6St-DXgJajd&#wd0kbFxN;;tO(<#42Vq zH8bSf%xOg9$-Aahbb8$oLzK%Z%1$<;*9=Z%xUdcm7cx7U(~?;h>{4x4${X_2Ui3WY z@K|bQQtw6MJXxKAetG?xZ}!BYKBPC%xszh|%A0Q_<3M#SekDvnV)U#bq`rtWgG(0k zW5Ig$i!otRE^`z*;@Q;2QnPCB9A&;;Z;lFzd~?m(ptHJBaqBb+)9^GnC6|r_S|E0K zYS%n=s=FwL?o&c5#^w;1RG>&Xr!l(oq$)#KybB00K{S#gnFb~nr}16paQ&(2g@cJZ zWccb`pJQo~k4YSwxG>S`FCGy@778cdl@m2^=__|jDmer5#C1(MuDOktyNSQwrM8O) zXMeU(Q|8I3vYSSm?x~j>)LO?6J2Ks0)BrnsmxKcg2ZzoZqm?7F-4`)Gd2+3*(3lX$ zb1J4eZubt(^rbvsT@Rw|A(E_hDFfz7Z!M9(GFIZkvz?_oTyF1yAia(oB_b5g9vID# z{79>`l0a5l>IJ&##c6_(%`!8>Pvt|Xt8O}k;G&eC!%9kolgu|uTHSAY=*mTxlgGjp z{V;6iaZGTyCnKBFig`d}*YJMdoYuWotBc7^PJy^j3=a;^GvDiUh=>_Xdm5bEEDxsx zL*hah>9J>;y2vv++M6w#0}s|Wado@XztSWmB-EJ@tyM};UcdY-L)vdS9A0S#ZX}*; zI@C5|E&U-VZoSe7a(8Rd=h`gX#Nx3@tNYI8wPHgUefT{>*SJt(UGS@0BM})5!^u*GnB@wu3J&`Rl?ruT)N}X3*a>$j0GgW`T>jelj6>uJaF4IBoi&lLd%J4~G93Rn zxtDMM@NC>L@xVOs?m4|uzIGrCzqLxrD5x_#yCiQAQI??y3iYC$whcv1P4iZzSqRJK z?rWWmi!OL2V1H*C4CYR?d_1i00?~8Jgp_u&}Wo0hq|T z%X8sB0XI){N7v@X>YEMXM;sPUMP4qj%$}K@*|(k`eNrCzNU=C#XN^k+T5zNY!eg>`*Kxx9J364^(-_1|N8d5O zLv>gG>k%)YjN-JHHNSFnOs{1<(Pv8xA<=CMezsd5;VSz24I-D%>+A8nmk>(ZTQ}h^!4l4L#vo^Ka#&H;(St?Y+r{bC`oxAw(YzBn zo%P{pl^Tl$X$LGr55K#1g1^~d?K#MaQC2Jfy?K?N><*yGb(K57i13bIcpR?!2hWx( zt?BpX30f}={IpYZSm%w&??%*%0P}~Vj_r7mNBT)(1$*8q&;!t)I#&H7i?U(oNJ#nb z&G_q$9Im{csAB~9kf`WG>bUR^5&yZyj4r#9GR9>J)&6J8=F8S9z~tO|BS-KbrU~^R z2>{M4132KRON0{?3aNW4L+Vg4~y%h_|` zcX#*P0ETh5E=GpzKkdg$U>VtNh*_Q4nZMm_-&}z4c4K%ri2Cci{Q4FD|4{yR0RK7R z{_hMW3AuMy8tbn^e7jv2B`rQEv{T2rnc%0BHAbQ^yKm?Dtc}Gu>bJ%6ap)y zT<~a1GTsHX`erX^i0)`<7`G@yDb_S}MZTYHF@zX!>n0(-clswYnl0zsyR1}AnU=65 z2UdufBMSC1D-a2v9(eM8yIwf}vmJ!H_A^p2aIj|wNBhuG04TuH9&O>ar6>tVvJ!Aw ze$6ncYWKL}Q*zo$pZk;&s*1A>Tk$OeSF}$ldsX|V)h^$*6eOXcQM*MP;NVPsBALbM zZeuN4Sa=MK;a{kJE$vqiRq+ia&nz-eUS8CnUo4Y3U`O6$_3yD(c+pl*-1DS53f;zr z9V9JXot8#RRjuth4Qe;bkM~>T3!OHZGwJGv%q_Tv z;gcb=p!O6ht5q+)6*Qk5+RVOzIvW_Z#hl18EkTRfE_qQo^O+9Kec%u2>$pV$-6Y zPMAr@_Tf#!wbVBz%N*$0CJraK1QNeNvT)ny&7)aO;c1a%P`1t`$U97vJ|y>?X-=Dquh^QG}IV`Y!`Qa6ROnok$bfzh>gBc?CB? z#r6u}HA4MOK6L|i$LPqzdQ&5TfUQipUaNvcnf9fdJv9S2NA$RvnGHRU@-?V>UpY4m zM$0H=ZQUVW7yqYn_xie}wI0I->%8q+2l}?L<-{(l-l0>x?=MkFzA2*w+}q8ylvVxp zsa(ZBIGwmi$SDlxI4ZD(vv%d*pZ{ygAO35}JFNe+;xC}=UEUIGi>Ni-csr-!J0fSj zJc!n%u7zo-{?}q(JF4pEx$INo(M(NgEShJ-*U!^Q0!3hEtwO_C4V6QHGec6xtDsOy zzx_1Tmf z&pv{~xR0ak>Tw`AjU1&(Dqno|VeBO$Cc=AwOP}EI=LTkClEb=#rCA;2f2{1udStO$ ztY)K=sK|I)m!yKclCzmkz@aMuK8Z3ASGnF9E3j~SD(wNm5rs9(2b-B>=t=0KVfbSu zlDl~f(N-xf)gG9RbsKg=cwQhkpyZhixY;`i8=BgICNn<+>qO({I#t`)&9?om2QQkr zZioHYgS!{o4MD&)ydssLS+TWG>s!_!{B;+8$7DGF_)u(mwXZ*I9rYm7lU5$z$(cqI z9$$Yr`a>Z9fNLI25k?!%dZZ=6sYs@?My(2?jaETk$&yX4qA7L?kXWE2O$<5q?4p1< z0mjXnk%q~!RH4QS%W-xMq8WaQdRuQtt2ST#8b#c01WDUVtd&o|u0G=&Sjso%ngo=a z_iMd=^G1L6qp;p^QeqhEtB<87AV>WAH3h|7SiIykc=fY8#;X>PcM&!DpioUn*y*; zuU>G<(CMsQR3N4cQvp}jm1@mw-p{E4Wvm&TuVfDkr7}#i6=Y<5np;~tff5v*x8qG- zuaSXIUs&FtJT6*_6G(t~UH+l2L)iFPa`=f(;w^Gs7}mmtn~!?<8vRiUB+Ox!fMmI;y;Y5Ur|Xp4qE_;&NB9b3jhS`i}Y zpJ7n^H!bI=B#b1wA6I;AzO{`E@adR^q8=;j5FRi>>BxJX$a+h$ylSby&bJUqgy}Z3 zS)k4)Ds3@K=4}V@PQ9Y$f&Bb07d+M~uAlw+dp0uOVpMAn6<(*0!qygK4H6FC<@_0kz_aC@_Y zu8BzhN)3bQo1bQcnG^z84M{fJ zFG{zT0pkCI=;ur+daWXeAbp?qYy1$I zxtQ&VE1Wonnn*Is9e(BH=?)bL%=YQ^DjQ9ET}cmShv|sPEJvo^YV$pJ@2o=Rlb#*Y zT44YaA|qNzs=L-y_tJKTTuZJ5?cVCBzFAL0jf0=mYqj!_L?H(rxK`4B z&Rbl+$I*aQGHLB^Ds$SN6cX~vO&gTXWozaZ5U{(pJ4Qjcn>#Ljon|jt|>rTSnyvB0l3Upn#Wk-(r^LVgRCWq1qlt zqH&W0a(=k956F@nStT=kZ0_F*uG`eh>x#}NA$uPjMJ9Z!V~_B({bpc%`=xy@(vtbi z6DM*g5xPa(>k|AGQo&UHZpank0b?y3bIh~r|;DfM>)x(se<_1i%_~Tj^hn$ zAib8~aOt4RoR>Au@(x3ypB2wcS_! zu;F7GQBhGWOxDEtl_pp zqOx^nm2%0}#>Q=ni>cNV24c{}Q}gK8SI}cu4!Z2XzEuq-sC7LojteS@Rw&RA+}Xj+ zl1hFSN!DA?MwL~tqnGE8Lu-ze%1)gF1Q1SD{qEA4*EjR#+0eW$hz_4r>5FC2Twk5& zgf~`J7Kg=zz)MzE8@igu#G+_9z}hZ>eQ#z{{?f${44;=LZu|NMqhUj?=_Vke8_y!@ zkz>NVKeM#zn-uW$MAC4p%7ebTDJ!46Hzq`< zLJdG-<#@T2rTdKme}pqT?6GSA9TD5%vUvjNnM3{9`aS>689iF1>o8hnQ=_K* zcq%_?+3pIUiqa*!Zlz2fF;HlSy@=N{L_)^Z>FA=ylJc^?F(}X@z9BX)_E5ToD-(e> zf@MNxxlv|FN~0v-4q*9g?SyCh%8sjH8LHphSg*l1;nExh^Z{{joXnO{&}KMph8fU6 zBWz7D?wD+}4(}}$G*=r0Rx{0h$27kDgJEkFmpFDjvE^&ItKsk_%tD>I58T91*+eiw zGjf{!=Q-$bS>upvK2HN4-)rl-kWgcl*|5aHXj}Ghy)_OP^)K}{iL_wympC@9E5K)? zLks;dTSdPWV52xd=v|T)eQl%eFw;P;(lVm9_*K$)J~8JjAJL>on)S`*c;2_##6?7W zuoWTm%lYngCOs?bbv1K?*>~hpRnWGzP8F*L6s;dD)$Vd}u#tr%vQ|`t<~v)6!`0?kMCSfXbmVjN^u!kZ9~H)LKj4-@l5)X(Em-@f~BvU(>9nQ8%I_xi&H zgJmLwT;VgSxb0F2r~5s)2JA!Ynl9o30xyk*%in0oq`B>{jf*Fg5M2+3g(j-3bS^P4 zPK;Fgaxyz72ngaZ`dYdrBpc;_Q&%_m1WlvWN?_8KN@Ay%4j`hr()6V#`NM}7R(mjg z9YR(sdnnfQs)V=A&oGFMB4qTmP;ZX_9v_Q#*!sulSTuhfQbUdo-VTdq6QawQ`&nF)IzczRM^X6n;(n$C_`+D3_HSm=5zXf~9aE8Y>7q2F6;WV8&_ z+ggnSKf4!O&8PvcYTuKE0co4@!akCchl^e423b|~Zg|e{*1w8g`ZFK~5>JK;UH2vX zVN%NwlMhZF^Af4uF>Gv-s{jd!dN`!KU&xc5f7cvjK?$UTP-YQcq;+21vf?vBjv zL?-!^6{KmY^PO@oC$|7`0JY(p520*?&xgBgSfGGwWnS-?+TdEOGuj5d<_;hAW^zE> zV9qb7JEgy4LaSJ??nu*H9$0ijD3xHQKk~EwpuNDerlD=o!a$kZVPjt)0VMj(Z)m#3 zQR?I;gTr|Ub#}@Qxt9#qTfpL(&1M~k2L5?Qhm4~frs|5ZYkv47U`oyGW@N4$uy7d7 z3y5}$n2DpK1JNnMA6owi1DLUM8$_{{NHluvZ!n5!U$yF$ z=;v5~v~?t^r8B%ysYzQrbQx&t7`;Q-^4JY8r~snB4YM%(lEe|sEws)_?S|JpN+`Jg zoWuf6Kv1yj``sw5JXU_;f5%2T6dC=2Q%{n7+3O;%D9Or z_*HCq*6fMy(#&k0P>@bc{SD#arD6~j-r1X|+F`U_5MAmb>0PI?#z1irN5Y0=m{@5+ zdaOa`(+c0pJ!uIQvC_=9t;x=V2ZmRWG3J+-fwIC(1ZT=Te(Rchc_OO;9;c)~I2X8| z|E`}v`8N?z4FJ%~^Bnen{}pc|YRJ8tF){=BKhNZd(V=uM0UxI4M9+0Y+r`X7L&i1q_Vb~Q%!QCO@Q z(mqaeo_~H%K@z}2>f&0`-bP}M35XZ)eN5SR881FR26Y1vPGh*b(=T5g;tg1sGKoJV zIPJzVlt75!>+63mHuK_S2pK|Tay^d5O4~D^y?6c_&%ehX)5?HyR#tkB2?s=w1(K&+Y7%;%Oe!kB@4B6(bG619R=y+E|LBn zQxIF88y8FR4@V&dk6QRVAfY6`ZrD|{O7Ln9h1^j_I)nXzzdpb(kBq!L24zwo8wC8E z|6x!5cAt8Bj)Q5M%$yPWuf_QT7c3A-q7V>kcL#pwZT?J`4XY7)2uFA=s6YH$Q7P^dsqD>6ZIugktm z1Er!DfX5-0{KTq zK|e9zu?y~ZnO1a07DTOU9*KdNa_!~bAMe_}jq<=r>%xVfW0{PzXsgV!G*D$22T$bj zkU|_9mPjOVKL?RS(tJV|Y=Y}W(ZpC@BxpaTndE9v327TDbjjaYu?Z$8Pl_U{+pu-c z&!yZTvTT@`T9-mYJKPxFe}B_tE}OA;W!m^~T`PKOE{8@j<1oLXBiX||gxDQEHK|jD zaqgj!q%8EIg=LsS2>gpF<_Zq*dAmYh?t>Ch?;z)6JnHOv4cB>z*RDGqE>M$^KXx5S zA^K(9NB0<8@%a8sm5x#%LkaL&jbtcxCd1M9jhp9UOAPgT;A_~%_cTjqd^j@U*nt*ia z5Sj!6g!*ng=iaO5-uD~dkMHL#7+&iyN8!7fVR=5e6nRg9Q$5+QeUQ@n6gR zrMB2`Dfd2_NFA7Ln5wr6wVkarx`s$NMx|EkrTKd3Gn%0|nHJe%{x^|pA(OC{9iuP$ z5AXhU6@O^|#2lxO7Cz$2X#dqMqWpdbxr!P685>Jmu4kGFok-boaRhkS!gS_+9NyE= z(8#tlALn&EuI|VwQ zqwCx|vQ*@33!bA_DzLMc$qZMPa!gmx;;U?ZqLh8Bt>(m$cUEH@nlIGt8yC%|s_lvf zJgt%sw?~I#E6xWeSHkh3$rtv9UJ3vNw%SFK(l@}bF;K=^0!Ws#PRqY5TnA`>aqbH~ z8o-nwGNY4XD+d9eOw2}HPcb4Z*H^cvV1AWu6K;kc%Uky9z@YXQ&ePQd3eC431;I5X z3a%(3Yj|dOfkIrLcJGGTG7V?>mz7}(rmS=?$)SSBEdoXXyFRLbxH_{&ZaCj0lJK|K z^gZ-u+Af0adibR9Xr~6;i^azs6l1o!PzKotWdT;B#}+ZqydM<<^SE zg@@1V*i}tay3b#+{H8ci{836q@m@#%DZD45Zso92&~U61+KI719c^6!uCm7B_dU<| z^h1~H2uh<6+#hcm8y8Z?gWm?c0QgStafsA(*os2sS3^O;{T9HGX)qK2X_^6O)%to| zvnmh^0k-KYDi*L3;Fzv*cG#SOe2=pzI)i^00I1lpm+tNr^RF75~pQmB!cREn&ItrM~vl$K(1bi1sVX${to73f=Y zI*c+i!)mN0X!-f4U^!YP^||e6R1G+P#Y9PCzU-CEDm4nrV2<}f#-ZMO?rABgB4QT4 z${WKnG0aM=F_7w@+7k6HNTz3`NLpu$^L&)v?0DDD3}_#Yp-aifS{9ln$`834y~cax z(`qLcFeYZ!!M?UeD6m7(ZI{n47AI(spCIJX4|n!BSGV3(&9?w}!dQc!Ob{TTIhX+1 z-J<>c`SVioj=vx%f3RnF-b?O&0)tfJ8Z4E0^lv5EUV*bxoUE`)EgAZqB+@ou> z*y5DQ8c4d|y&GR6_4~c6*v#;HlFZOdtVKN6x}E}7GdyQ5s)>5$fq`jUCl=rpl$#=1MwpuP((SMf<++wHW-^QP#bW5$7F4wF`xQjz z&@xFaQWTgr5gPaAKRIeh)+Zw2wao5gqBdLVldH0^&R@?y8a%vH?@-J)R|B8v*yn-~ zrrD(UTxcKLH{}3~$Hr%?{6KT_uWcOQA$%O z%_1U~pEmKT9h?(j1)v2*iYs8Er6(;-zqb+1HGl#}TX>;Imt$$mYARS)Uy}mymw&TD0KahOJS9%dW8zDmm5N#hMoEjV`FdXfn^+tu=b?XB3g~++4Uc zCZnFK*%oc$yv+nJ5b-&!v{Wz*oS)np{5ByoS!NS?gdC`#208KTmhfxz(MWOGK(3gt zp8S}<%N#y4vO;#bE%44AZH?6WdQg$xoB=Vpkt|)f-{|z? zP$mPKajHXDgTP{5OsJRqxAm?@%7O)b{EF)QoIiw#K`Pk#HTJhaljn2=&rhei2RBWm zLX#$oV0_Mb1y+z(nOTEp_pK}Q%>;8N{)B5B?0%~Qu^B0*Wor9Dj*+W)Rpk8(YvuBB zOhpA2ejMYst}cenJ=Maqx96*%{Dk(0m@K z>mUu*S?$Z$YVy<6-Pp99EWJ`X+MBUC*XMQEs@(DFD7+-7jEGV0u&%imlO6{&mJ17f zYc@>M>>Z%q?X&vT26qm^&4it_RM3^cJbm7=BhGL6*pFPXq;j*tWw7qE?#4^hq;x{j zvbEo&ntu|?6&Px@iB9w@WIsZbyw&S68`!uv;al9~+0FNi$%NR;PHPpGqY z+1gMWmdmmsArVc>R$}uWRlJqjEYyGYj;nx!J!`_qb;2#}`DDs2>6Tid*Z5XJ^OYKc zFLtK{^op$>nP5NeA8@9mez?D=l9J`6ME*M$q*eQliyxP4Nln3-_-TKR?wrBS-ZV2` zMRTdL4kE{(x`_QM!rLi7`3{3(AU33Jpb=K0pyGK@GRcXjud37_XnuG=w}@Ws-b-Td zM}*4bv0IhG6+Ua$jF>RZe-y#n8W|{Q{GK4)1GCz zMRP_K6&{-@C%Z!MWxL5 z@6NT;$@i_Ms`I0gyvF5P?3;Eqvjm1yOxJm=FJcD0h$rTc6Avt4daNy%M=`cDr;&~I zC#-R|ODKWvDJNMSaH^BlvSach5;A;s?Mf#X-dM|!j9m< zqc$W)4iVV(El|1ip`Y_ihu1Jgx2Y+%A zC)uBa%0>{CYgLRY8+et70u-Bn<@2IBYu~+Lr5GG*CHa1g7d}x&Do)qKo0Am5p{%cR zmkJCUn$H_-f`ra{C;>ktpgCa^h@b%+>egX=s#hr?GVUy>e8ss@gZdS_sl#>Yg2L!R zH8nNl39dx3>z8d(QO^MyS!m0ybk*8Mmia{=&;z9MQi|&iii}rE;VtCh$r|K3JdNAm zzspine^_t6107q*)z{2BW*SX{>G5nOp1rV(u4BVr>5@vhn$Moy>V3DKTbR8|SD?k? zI;X9mhDq3Bnkwjycv_|D-G(WZPh*CtRL=0bEUViF$^vtx(b24~q=Eoexlt;$(acE# zxIWOlhjFPd>(stB<|;p4@N<_nx4Y8l%vDbN6S)LSFzt^Q%?Xqj*^=D+EU~|q(wF;# z*-tl@zWKDU5c%U)WG_$vz&6I*v%w3Hftr}%{HVfs8{55CKU>lN22p;{`it_GB7=vtK22shmj4k*OC404}>JjJmrT~{g{6@8m2+q{J;(s8JAhVyE+S>VflXVznt zWTA=Y8>96C9BGLjeNKIi*6Wi-UOz#L=fKpGy{_ISyJ>fClryIMOAA1bzRgvTfHet=;6^q{rJ(FD&p9%@mm zh@Na1RYM}9>&=_Md7DMi)-{GCYM+$q$8g(`946j39yRLpTDLNy|O0G%g(0hf67(z<5mF0 zV`00o3yEC%PvG^3T4`Pvpg0l9H}IKegqxcrtID6!{aOfoW*DCe&je z$7vVPy7Dph39q=cm-NWSW6>O82{K_EKH{lpw*QbkU!z&@jdb(XCZkKh!Ck@PP`jqT zTLrBtKZ(Ea$bS=mdjf6_O<<2MaJt6h!3_ZA5$F81Vq2dWqwcY}?EGf%q>9SD6x@_j z1$#Ov*EEz%M>y2MWMT(qH@X#!c$^4jNsRd{N3TLY{CeQ%4YR8|UgW@ZX-r8ejg+eo zOYjVD$&A8Y`YolDLp+rvp9tHTHj!U5dCKp0H-rnv05q(}*dj0O$Q?x(y5m##P5q_0 zkD;)jiZZI~5Xo~*!UST*?fO47HvjZ`XGH;g#P`6~-WYlqtey~FDMUY4o1#DaIFZ|| zvDomqAz7MHhl+}7Wu-TU6SB+AUnuRn*5P)vt&2O3n?wV0Ueev{&sS|O$J5eKAmjJa zNhcSiy0Du6z0u_R^tJThhpe+cdgW2=qtG{Ma(}@H+ZfN{O3CmZ{$8hIn5wkK6@(|s z^Fg})BClK@@z29vk@|_B89e!a+bIG#@^rD>w|`5N{|BZ`3Jm!C&uN8!ycgpMfK=bW zl_!4tuOs?-SpVan2mpWz%aHanz!~7rU z_f2U4n`wO)9r%C!&cA=~h8p-|{<}H(@00!S-@ic(FbQA2kh|v1{6C(_)+nIW^Zz$4 zf4eLHb2@>><^Si)(f{Kct9!q$omPpPWjRaPD$pCz-kZ^iDI8ygR?Cc4i807|p{qsuY_Re&Z;{1HI7doAmWUXDa@TJ?S7 zHmkInGqQJpB!sE)9(j)&43PX4U;bXNt=$0Wd~D&cEYh@9oBpq93O%A=-geQIhdfZ|wUUmI+i$wbyi2E%#|CRRqkI zDwv1kvz61UG??vHB7j6EP*}Y7@|@pN2ZZ0wPsXhQ)@wC)OI1~4PIMLk=>ZC3RK0|; zDics})-IHukGEM@l}M|d^?L~|*1u|~Ymtq5;#d`Nu-?qX zAr;zN2wM1{Ji*&kSJ!xUd4F_OC4*B%nLagIR1;Y<#mbuV*6Gw2K1qc5*0R^6p6PKf zMn~8ck-Z^WJoSf{ zPu=yQR#Tbbt_$TFExT1XpK*$*rA1g$YR(xKpqF}d5gGQ!)Q1Yr%c*LqiF~zGhemXvd^ub7e3rYH>)xI^89-?aC5$nf zeK%UZn$faC0T&V0<1KX|ilZrkPLr}r1kPC?u4Ud3a68L%jZDen!UG9T*DgspMIpQB zd_gzc7>lk5MGb7>WRSaMR|G`;^29md-Me@B9a|-9$qfQ7TEoqjuBm*GV1vmGX;=FY z`Fy(GyA;h?v%J@>g0a}7%W$&}8&gwb^icy;^X(~a(IS?0hfM=7nWj1_TmGn-6)?$& zdmI@aA8rjRrR@?(zYG5`hbgzN>BOfMZiE$YQwWcx@I6<)hU=Hh zEYp*u-BaX+ws^O!f>0O+!c?fkP0gZ>4HKvBZ)!GkR~POL8wC*55x_|sq(}>MsqT~{ zjMHUr&Xmqik8YP)TD?3Alpjr8x^B2>OyoxmhbVso#B_A+?VXAaH+=F8Ksm`kVwf%$ zhEOZBv$tV)DAi%YBIb_1X1u?ZKTLn_`(l^;b;j+xcl!H|;}rmhHnZ(SIVXUzl`Maf z-T2HgSLke(0qwKHK2+HKlZN@PZS{vEAewS?R(!}Owc&7jf(;EoL_Kl+quuyF0qyE> zkF~mH%|IH2H28s5FK^nTv$Y0xE`zZ~U7HGmO<)$BE44!SlZ?*Mrf_T64;a}1z&w?6 zLeg&0?rTrbv*JCr7moL(_aWLG7WCSaC+m((vyOgS9|jZwRT0s!y@aNtbfU+FxT)Pv+Zbl5pu4-<5)y2jc6*1P4DB#9qVNNnb1v^Jo_z^Yft8?4qO zV&&viE_AIsMLryno*$hF0{Mt=jMLi;&*)Wi1<$x^A`c2|x(bkNve4!<8el4`vDqVM z*5j2%%#)qTWAbaUG?zV}AkU-2-h*tPh6dR%Vyw$w2FqLh?#ylq0`hg#PybS;5jfp^ zz4ts=?Rd5p^Ium0*bAsX$*4`_8!5~i)d>a20u6lAjdto4fW;d>+icoKDIw%pdo>a~#hZelxU55wn(PB?2v%RWA^&;{PRTl3wU#1lfaz4{g`6C+pb+H$M}-S{+Dcq zer>`g#%Ts{v$wgAhR@eAzWAPlVR&qgf>xhe+?R`ud}7~vfGfT9?DJ@@>}3A>&j8(V-; zke{a5Utvr&?YH<@l|hmN{z`Fw=8-4OZ_)|)ic$oRzd^dbi({u0m?f~OGz3~C0rMBL z6)|Rc$(OEx0Ladqb;O;50-f~%14{9)Uy&mtkP>Q+W%zWOSCf)3PvkrTtlI#%7S63T z?SLctzBF~sH#57!G>#woSiYl4SNfO%Kdc&7@|dO}OyOw)*#ph&&AC^fI*O`5sMvZy z?ygKyraRG9)h;7X@Bmx%BvDQgu?#4C&ZL&=l!a8*^0@B6Ijmu8*EC#hFgRbjKQOd( zhLPt*rAOJ)L?)AVY32mF@n7O+gR&rDJ=b^gAqYl$2fGRRcs6}1cULRw6kfY8^SP7w zP3z4f$iu|$LA`=or&!Sw*6%ExoTByg-BP%lcj*|(Na(=@k4uv5x+BPW>}m|O3r&kR zs+4|JQ4(uy`Fqg{(_SxH)cV`Y+@_3&$B`T+FdxC)|L|6RJiehlaUUfKLTN&;hUYz7 zWk4===WC54FPeEU>n>w@P%U7WVPvsx5HZHeBBR-X0g!5 zOkEc3e=2Euk0@fKbCIP9xWP-!uD(@EbuJc%J>0n!XL2ay#vD4lZapdfZB;?9E`>If z-pC3~EU89R`W#j?{>am56~v*^%}l%5pckk!O2B3ySp~trvGeu|Al$AgG#o!|cy0Lb z<<)uykb5*%{~?@f5=FqOP8x*W%?Pq9VfF>kU3<$7e&GeqJ_KlXQJ$FZRN`c%=QG}z z@Rygy7$V6c*hcA&W!96WXZQCW5Px2`nl3siAB6IqA92;Y`)Dv~r7SO8U3SWhuQ9G7 zj2{rqFEkPzM2d^e(%f}J1U4YfI4-h+1G7mHXS~)cw@W&jOP9t22hsw4{PspsASRGZ zhEV{_eL$hRBQ6N9tsP3_ll1i34D(+P{|<~FjaC-}eFt=5&)9?<=#E(B4N;n5xuqB1 zZGUwEQ)j-h+RoEVGRY<`tT~{-SE9+4;dVD?r9T@l6?uw0aFvE*_Z6xai;?hI=i+DK z441!-%B&ah=<1ek;IdH3n$$NeyX@W@sqF2?xM!4Itp19_XN_hq*I+7S18?L`o-)~F zh%AKj!DpsdB7P9(*+LcxB_wT&`3@7e!-PDP_VmckyJ=0KWE&52IG0J;VnM`#xu(Uw zkcHHFZE!?l!aFxUzRY2Pjr92mc0AjeBy$xF`KDdYA4XMP@3rVMo;xZs`txO$_{q3v z+Tg72l@Bi$JG7j7-Ra;dW%xiVB|YCd@SRb>!K(o}yCcu(lia720cPmjR$%hotSyVt zOdWqhORdh>TfT`%`Wmx;35hAQ{OVNl2mWr6CX1RJ37?&qpBW?MbU+R>r{<0%TzRQb z{`BaK2V18x3ko%AdRxFnw;{AZp3%y(YF)208m%|7IZn@dTRDjvlizvm>x5kkbHY&7 zGy4HkGoD#RA3nqV`9y5j?a8!;nq^P?8MnTUsS_|%nc%`x{5NwOm9#&GQ-mD)lq`KM zCliDmM1uEnLscRg9Zl_IhnG`SI=?UCR`LG$lK7idV6o=aNUlos9NJG`gHSo z!WvmLReNJbmZuHUd9^3JU%MUW!)r&!_bP4_fYVQBCL1GXb`~qz#3xybF{vcnj^k5W zDCv~*l6CFsx#D1_q2tQna~lHl!2`r&j z8ZRD^aw_deNHA4@M4SgB%N zPPY(mqF&6i%q(TrnBDX7sCq?4O6rACWXv`r)ADeW!gmr&pv3SJJM@Z<&E#wWT+GQW zlDBOFT4gHo^hzFU4@7tDlxIuGT(%3NVqKr&lT5+~K|X%FRyp}Fz(YcVSsJt8q$-oa zjOyQaB=E$)Qnq!Pe_%qubN-!(4YJW{>!DoADsWQ{`m7fIdQ$?=$)j4+1l_+thLu_L zIQJgYe$khI{Nre#H0XYVQ|}X`y=}4EtDW?(N}XHbJbmm2_4x<7!WG131OIee6qCpK z?#V@KrB4wWqPAtABmdhfqxqheDaW9+UMmnDcUJd+f{*$VZ_wP6T-{jTsQBhNkwX2UG64ylzwKUC2 z?0!DLeSL}l1F~@r>B0Xf=0*E@X@#Td5b);u4j9+;6fAwOud92yjg*o-SMyxeQi zpVW*=j}>s`9a^08q5VD+mdGu?l7mm`mYb@T4ri*PNcrnlp0+kAlSd3uzK85QXgEA^ zUJ(%$k+6vRNDA?xM$-AUyT)U!3(2+0VW@H-Xh=G(=tOUzM*~7eSJsoqaOxr%m4wjU z%j9&TGQT3 zGRg)BxOTs}lqmZV^N8DOmXr@vtf$YBuwvVKAl|pF?-ahYwH`Wh@VS>Z!bzRu#;z;V ze5z?W@v%UXf3CXY?7C(>hssO6*)94%;;7_+>zvXm(efgaHRR?$7Qpq(@4b&}{TA*V z?=93kd3lt&*^@4O*L+y-Lm{Z}72qvH#j<_#1$*i^8t~P9O5+h_&tthfn*zyiS!BzWX%^&?cUBDUnb@&(WAv+}wD_yPd>#FV==yXVG7OpirH4 z&K6-#=KpEKqYL?f^?!W_aDsn%q^A9WpEcAR zkShASV`(e(V+MmS7|1hxkL>1c$>0CVpNCj|z14;ejI+D@ooal4@au=lQo7>)7)+oL zPpf}=v}>rC!LO!rlqOEC+HBBy8yDPYD#B}r#)213Na|ljFz9@G1W*D4XcY*T#&Sk( z?_jR5e~7qYs8(kJy7r=E)@JRV_i1ox04{fy%CP?rB(e=X;{1fqmP3KG0ed>j5VodVJDzyvh56On;hEgM+7mmHO zlnD}cyj7nH85-wD<0b}e_wO0>X-8l z#H@kjvOg$>&mI6)=wiW{v=8o|#g!jGcV@!0^VJR3hT>!x*V(8bN2E^#y{H<6n=~)9 zV5O*=f!IW^4ta(Z)3`}MMVj3ccKWKSy3ZnrMU1o|fIYI=oTP`Br^#_T8AEPYI;Ug2 zr#7{Nf;8`*txlac8C|0&K7OC}TPQSRl%CN!mD@2_E5v^KkQre729HwHS@Ki)oqR0o z`S{haw==@KscH64eM& zr|mC9pMQv^l}`yd`G+l~*iE;4KN^9D)iJ?n9FY7CF)*@PawOyF+5UR_f850&6?S_P zn!q9k$j-7(T#mDg>KF{8H*gIapM(SiVKXofi`NzvD?pC@<}Al+)wrk~`ZSBwZ>7uY z0=~7P*D0CrLgNhz<;2e=N*CwI_Q652_CV~m-tq3yA{oEN*>n8b+S(;Rz=7`Z7y9Lq zsDN(;#m@oOl*Yh4Biq9$#k_}xWXV}Nq4RaPq5HTpi{C%{Im(u(6q|M4v5V`2!;;w< zLi^Lbl&31t477W|6rYf%H$3&sWO@KQ4rG2?GT3YT+v6}IK9~u)$zZY^BTsiYg@4P{ z6sEhod(KO{PcxNPtI)3ISFckAA@{7|-bQdGDMiHx6Q%kkwL5&Vy##Lu*-Y8UX6}Af z8ARu0v~&k9_;MD7_jh$jE=NRL$pc(f$%IO(P>CbkQLtE0X>HK}@T$asVH{Xj}7I5z?v*D(8m z!ve;>`L3t9NUF$v|QP@#s<5=p8 zS;(WUF0ZoS-Td5%`=)M%7uBn-0iEX#b#CnquSX4bq>n$S3-u>=?)(_cXUMTFiTLp0 zxt54X-Rsppvscsf%B1gWcjGEi!-J=^As|Ta_&CYa3IeJCr%ni9yZT^*?@3(Ti^&z7 zDcEtR7gm-29ANvX+Bf`gV;b!@?c;c~Et1IV$*XtP@=m>vShEYNRYfqAF8H2+y+*ps zg!ac2Qc5qFwOlf3X9;0~)35S-IJrSj-+GXX;$j|7QN@9fmTs}f^Qb< z3#qT@-H*-ngQdbV`+8@&T^~)*jPmWw+HF4!cIxwWvv|)18^ajzClA)a6Is0qgIG4> zpQ)FMojcQB!%9*D!)z88Un@y&GVFWy73$04F9Of}z z7P6vQes^@nc};#($f0Wo|N7)zA{#e_77S*%a!g>nBon{w=VF<+J3`96>q1Vf;UTx} zm}8;Etavqh-vUvhJ}IB(HNPjpF~4%HF&B-I0Gm}d8-a})AD^}cuIXV<_QBe&QMi0u z8C`qhyw?0EzjDgu4fzDdpigf~J+dI|=jDqsB`~}?hn5)ZjAPql)4T^q! zy=G>D1(xe%fIu$4*e2zC*dPB$TbnN@b|_>lr_W>u0e0acBqW6S zl|4S%33ZNSmRFM_1PST7yYr>3ll-1000|8rkPmy#EAB! z<$2_&IAsI64OWxo{aQL*KUh>hJ?m=WyVD(aHVMpVd?S^|b6i5e;e%K{XD&P}9PIhu z(r_HOf>e}uoT{e-zNO_UcQdXI+?kFxp6}9|kKzy6ng_@L#zW^x-*RUsf!&<+dS66s z@wB409y6%dK8Y{U$u>l+4W(gULfvE^-32{7NW0bW3ZSY0HsTVUOcyFDd6uI%0=szz zVLpC|Mhy*4o`ROGsy(x%%$TE@QlUf|>_&~ZGvhMHjr7oP&Yu2QqrGWBkV=!+(E3h! z$8F))FI&{i)9c*#N|U>N&YS$$o6oK5)Lhkcw}I-t+Hvr~?`KNvl=A02X=Bnbd|>|K z1+YJvBDv+sMc!U=ORp#X8WL$fKDbSzWmzyGBIm`A1LiI(2rilgAg8=nhJ81>VU7c;g@!WU&;OQEqAyaJ{~*2jkNNp24i6nsNoK@?&LKWA3g=Ttl>JF;?!SVOjJh7AW|CBJ;x^0_a1 z3^z?61bHNK3BXJZRXZahkHU$$r6RWZ<3eWzFdDq&n4jmA45-CcFNR0}5I@J^-01Y` zt{DEhhUo|VovVv_BS%MZu?9-Aj2zwZ_zw{g6Q$Pf+vm7Rc-Uoslo(afg%B`C#L!3R z>PHNg!)>&z37dVq9=SaG&(?8m%Dq4a3`{LJ+6li6}^RhEj#Y zjT~G%Pbpd?#_8CpfeNHtKD_fVDaIwm$&5KuLyRz0z}0}6YNz+vyTUN^`OctYQTYR& z-h}hys-lrbUFN3vjhUK#R5E*2xpR#AJm6gLLbtxTBsMbh53KG~PscJ%8xakK{%U5VQJ5Miy9d2V>*nTp!c21DP} zcH!G-$VEl*J>&8S8A_SoWp;0fR?NtC8qZ@e^INFA0hnKpKsPp?D5mocX_g3x3?w5> z7SFz(jXyKjOuY$uSc=03GIZ=p^D0h8u!g?P3vwyn&+?jP6BzexQ| zTs`M2Q9H6nPJ|hxo`Zh(WuJ+WFL?J!O;p`pWud%bLI3@)>Tswz%th31H|t>G<}WpJ zA`-$7pKgNQl*I{E5Cw~-g5pEh7#TSS&iIpS=tQ+6W^289So|@{-l2MTvkWhjwdte+ z4=ZTF#VQpHv=volgWVMggb|(xD-s*;osW`rVZr#e18GloE+;eFhNOI3Za$~*n$OLg zP@d%0j{6s2!Wb^Bybo9fM&D$lV1mYq43!lDuVO@?qy=Zx_lX@M^fcB^iE zOc%qoS%0SA4rG@*M8WGy`tGs_R`-=gE1RRrCsk5`tG})-njh~lo)_Z9Ep`@iHmhs) z#ZXC1)AQ)YI4E#1P9$qX4dtFms+-DM=p(Z-sh&c+<=QwH<<*F`tUcOzXnBL2*N5pQ z-l2Z`M4;aK(Se~O7&Jv3wyv~M0pwKGyB{lcl{+n`XSMV6K}GUpQM49B?+iu5u^I?P z_R%EEs|rc`rs7?Rj7?qK0_k_g@up>VjgfvlSh=(9rU5q3ls5N#iRUCjuO_p($79La z-yc1jvUj`x8*=j-ibI3*x<7TwxwuJ0w@K(R*-l~#JH$yx=4Lepck$7m6tf9z~=g#hJ}VF){fCuG#k6`X5>DFg8V6`3TMD^*ZUR zV*ucT039WVn**TYa=7ZrMYcfM`^ANwon3PNi(e#d`!l?#`xvjG8ei$MNAPMV8E>C5 zVz>MF_K|#i+wKh2c*Gk-B(9+MGj?T-5&9mXEcs3Wv%sRo%~0`rjAZD)&D@+hm$9l3Gqa8S zAriA2A<>83O&KDiaoL9+<%zN6TqTh*xxH8Ta+{DqNa8s}HrMo&;dXkL`<2>SuL$r_ zLvpQ;C#79-wo)M}dge;U-UVj#>E}YVlPc+s)lbi`W!_?-XSOW^Z&d&@o|KsErJ{>Oa+hA^MA zWhZBJ~@{wnPs`l_%Gw71{d3r9aPGD`ne2N4Xx zXZ2^q1<2FKiYPq{1dJ>yD(bo0IfIc$Gx=!?Dq`0HYX9=;sYj_B8Bhn`=F7?)q20jJ zEIALm{z+f9E`wh7bGMyK%&EpE4gkQ<`4~PcL`tK?1EncsoBqs2Mrf~~p7^auS2u3WsHpkZ`0`{ax zC4FCsRPDHkz9RXjJ0P>(vG=tujNIa9H_@1xuf()9OLg$`R5CjHK>cOz3v6?x@@A_( zIqPDmPoD{gg3>kSwWXQ%n6+bQB#5<3b+B%b3A{Rp4?pp}AkJ#Za%NG_fAaQbLzZF^ z^10XHvv{b^+jg1DaVKk0{kxaYKPgqx8^2X;v9q9UKg0R|boC;`Lh$i+#ShHv9est{ zXji^v|0F?($CNKLa^6G&_*!+Z78%#u30{d+s#;+1t;((Kd8C_Wi&xOGj+A?ngn?{} z_u(&W8^g&_{$5WG{yfF@!rco`&n7Po>sTq+o3TvuC~IiAYx3%qOFO9uKdP<`wlzD& z^{v+G*5%$Lg=-Z0TQ090y7&|FHgrEdohFUa zB2OPLA_6F0WCJE;GEw}_1ifuWhPjGKjw>7DbIS8=P_R^KY8|Soq`vKa=*B6T;1w(* z2!<_myt|+++K1=gBmg;>e0w7OOZ5Fnhw&?>c5n4YXY5hYBOE`w1@p!iaZ8-;r9y!D z&f_Ui{+~Ob&hh>qg`D0!Pc*bchYMoNv#jF~-|Fo!?-#a+%V1GJx2l40R+Bn4Ji${< z7Tve@m@8OLli6&0d)Cshm=mDw?~d~_U!dgyWSPrVZj6*(DYYOjGllmeWT{i$7`bI^rt6P} zESEZu(9L4J@+dz0?7jIArs4@rq|ySO&i)w!DfKbdFglbR3;+z$%w-q!ImdHaMIIwA>cZ?x{6g{6#x~ zBP+-YsQgFSg@g=rle#uv@4)3!er!#K`>OF}eNX=?0iG#50Qx*z?WG2sr*j}@0;$3# zf#V1Eyp!e_CV{^kLF?wd-{3nySL?swyAdPMX^VZO)p=`Ch}tJkcp>afo;txp@@ee5 z4=)22_#9e;aLp}kG(=m@`y_$|1Fjq+kv{UbJC`~6U4~ZB#hNU&_S`^}?Nihjv!8&SNhf-$T&foNU3_@YPbZXBpAS*bE_$Vf za&R!%YP;)@x#3gb=-m1E>~RAy#+xeOtnSXNSNYs%Zw!}66C+kB#ZO|8 zc$n@RMU1y^^e7Bt}UtHh-xPh$5kwwxxqKK^gzfxqv5(5|pn8B6~Ex*hc@^;Ibyl0)bj(`j`)< zCtH#PFFe%>9go9NRqtusJT*fw7=ftBJwP0Bz3aP8f-DA~;~7E9S96f>EfhM3OOm z)d5v~NmgZ|GJ97>vcvP!{(kR!W^Fc-agonLy&txWzuPwD!vDOm{`cClNznTz1R^=15a9a4*Bpy)k%QA;FFupHh(C9=~ zesbb5xa#b(BpDgoU`}Gh4TzI+;146?Q%n#AH?X}J&KN(I22?j--WgwmW4dEaYDk&1 z3j^KAmQPhOgk(hDcp+hqhnn)Xg#0Vd{(%7KJOTpK?pKh%iLd#pCoV$VBAqX7zcGm= zWE3#CH93dGII5E&yOo~UpQ|h{vv`fbeSL7{!nxqt=i*zm-don?8pp|>E>|Y`P zVDKq6<2RR^JiQ>ylg7HsaY&>UPR~W_AaXqQR=2PB&Drj5i<3&)*Vq|H4}~sIX4`+h z6xt=9|IirRb2p?ouPQ#9Ny;oea;$se)C)!WCWViBf2%74mpgb5L@OR}_l^nt3Enp) zeUH8y8bZY#g?F)(HroPDvURu5m59KB1r8<|5d)5GwN~;4TAB&q=e8gKe%<_gS#XL&j8N{e~Gk4F`#<78e%H>^e^l*tbSka@GQ?Hb)*i zPShN-7j5KPVZHdin!wy#@!EgpYKIR%wi&}nkDBAJhFi{_xnIg8iqJ~>$)AIQ$(ZG4 z*OS!`cV=|XH@(Mm_NRtK#e%Rm`Mw0P;ig|g?^t$PhOU+D@A+Rm;cRSCH>qDd$N^_b zRcON`QUMc;z3@v_W~scvlh(FA*rQU;qPRPV=p0N+tpbys6m}@oZ7}tz#Gj2*4Ha6i z@BZl`6LjRiHkbGh>{sh${}tKDHocDJe^g_-hlOrmVE<=XB` zD~^DY&wZYBPpxi@!9>Qa>X&i8)d>6fM*kT^P+l$~DDm%>`xt3TwXLS8{Zy&5_C$X5 zMDCpqNDKvL#u0Yler|_?6y>2e1Aq37o3k?O3zEA}k4Mn=o?qQR5%xY5#$%ShN52m* z#J~3r(3H_-?yBDVX6^x!80U#q4zD^?qM-izN_2q zNeRI`0HYXJtyJ;OY)EsCK@V?x>73w~#Yu|&`csJa;fMX{!u@phW^<26#1gl?^3QL{ zaNb)A746y9f8T0*n$K$~Q+ zvDTBM&>avLB-4q|9;>?RefU1637Bogo~u8{j$_sW?c1mrPlZV7C9(A9s~6NqNNnxw zgLQDCz5&gglb-yhN6B%Xc^r35eWJByAXx8WtHr za|7Rd>vs~PDDndwv00Z>s=%ARM+ktURJzWqU-zw0FQ-1;S?AlGxnZreCiGT>H!$e* z@48Z!U2-Z1uh1b}yX`%m;!mg9_(pS5?(OA2i-d0aT_%CRCiUY-1%mXY^t9AvTb zqcj#t&~#OLcZQ!GXg3g3d?+BgV@YQy=C9(rp*lK^)6q>&hja2*VCUo8zr#u~D}WxR z^OejI=5nUvFeE)a*zRS^g6Cs!Gf13r>9E**i8VhMyvz{)co1_dFhdv zcy5@kWl!_}koVS6QTEZ>uObF0pmYc*-6$!IAl==d zbPp*Ff)dg(baxI&cPZVCv@{GoFm${(K7~)b@B3TluXD~?F4uqv!wqwP_rCULUzY;9 z(2t&6+cK=#FJT}vhXQ#AE>8TD3)o_;Y5A9xDWw{x-GjTkhke1XKe~{rJ25?}EOG!p zoaq%0qp+sDA6HjIg+>X?6Ve@Z!mi2H6s?S|a(cWp)zZk#-j{;+90c8C{P!llyvE4X z?D_N-&i^huSgU+tHw1tXas>>w+~Hqa_rF8vc$4$B?!Sv>G+DNcqDykjTsC_wl&NYq z_BU*`?KHVT2$)$Z)~Y-iV0f_6aCHxmvlacn8gJz(y97{cXlr~R;(Cue84+k>9S<&Z z?iB|Xqp*RY!dDn$)Q>yM=&w`?HH8zT%t{z&9U9R?6kY)=<)d`r;P#T`DF2<~3lfsA z_F+{6tWLFIUgi^ZqbFo&Ngfpp;!$)#ZXWGwiE&3)M@_s+I`CN_PX?~2sILh$5fs6VYgCVj9@WV!WjLhiAi8A>1lmzV}=0Q!U!kJSsiV}8PD z_LBBnSlN}4@5Rh7-K1J27DsNj*YdZ4DjJfk<}|DLPQr1v>=Z?6YqeKjE>K;oS}m0d zAK?C^Mh~HHicJcqS>RzZK!Bi`%>H3kT|K9UEx4fqy4gF&Lcw=axf!~3+dv3+F32o8 zg-E@#${tnY}=P;Um#G~J9WNLzU zcDVJ`^W)Xq^%9xK-*e-GMQBp#lp)%wk)3Id(*<8ue;JXW!}5z%_%#>(y;SA(mr!~h z1@DukKOBjt9?M-HRcsX*9&+BopA|Oo1rn)jNf$Wtlpwc@soa&Qn_Cx-*7nkK(zg>` zj;^sxI-{M2MD>^IqH3hc+-}pKl{<37Z?VY;k^JEW@Rtmm;e-GMsdhKc|7^urW{$W) zaiV1X+1L2(3g2C_(71MX;zAvxM^T7xReu~Hx?Z^A>5Z&AJo&-Pc|~D^R_)s-({!TJ zwDYcOC)L*>#YG#9+TUbD-)mc?BX_yHlxp54WVGVAK*xBB;eZW7Rf>V7fumAb9! zaT%kXg2Zp;y`NU4a%2GzJ!=#YzR!4t=!7j$`fUC>i-AK~Gz9Ufl0Pzm^$DI1HO;bE z*z>K3xPs2g%ot3w-eX`g#0Zee_4Frla$z)z!Wk@1OJq+4851PrJ&N-rO*tJd&E;!x zw;(TBlzhCkm<%G86;K=8&K?O=GVmw{P6J{jOEggW(tTA>?eI%owAvfGOQ&7Juf|sS zCLh~Ev2LooDqmB+wz>wtv~%qVjsmN&8Vd6-T@}AN`P;3hhGr0MkU%mkFzVM?m7)>0 zxFGiYb?yb6c%i~berfRDSuI6Z6%T#*K}$3Zb^+^MS5ut^RZqhn@RI-|`lcudK{33a zUbXg=avAuohR|V+k8(tkZ(|ws(rv#dPvo=}_h|FjKREP>HX1u_t&?s=1lq1df1~zX za!PujUL;;71Gb8G8e4~wfi(F|Gwvf10X z10Gb^?&w|=LGk&x6=BNsv3I@Bf3y+LB<#F@z(WjmjaUgJ{(S4`H0%q90p)hNf~7_{ zWv`nw6I1q9Q~mo%$YP7wkKzXzEBh2!=^^Fio02#5$KQ&CT^4{w)OJedGX5|G7M&M&%4vdnbMkkx;_Ya$(~_Y3X)^PA!}0OlmT%P2o~Z&skc zJr0q63uo8KInL2}W&{jVZCHB*0MG$1hKmokp`qcG!*-w$A>+-HMp8Papb$f8I+dbgqcs%Le;jCQwa_fL^h0y~Z=quq~s^o%zd;^~3(( zAzH?@KOouwjsL$xwAh5sb%QwNrrZIK@J7zBCVbUOv!7F}R0t7h|LsC&=m1m*z;SiV z^zUbpf}ffR2FSI)D#rTj=mW;j5N^?c2?@&%1{Q@^B5g+OdbYy!wHIlL*|MQc|wg!i`TCWnqQe7J8Ess z&!>XzG3M8oKhL^I>1t71(F&@g$D61teYXs=kJ-Ic1ln|@h6Z=xKc>bA}Z|yGU1Szlx4s3dBHZC%;Z}2RpK0#%IDYfyO%yr!eeWhj()K51OIAU5q3Nt z8?TD#k%Hgn0Vm9(W~rs}9m2C!o{Q}v3*M_6sdINu81st93QqBWG&zCPuR>wu3Wodh z;R$?hE119lEpf>`3pGxCP zAv(?ieg*Au+zm-TPtqffYyVX2K9W49Xk;Ywpj;sxA-n#A6wxGA-v74 z0jaMXk3W>ao_?zBsws208V_O%B4A~PWl5IPu>Qyedw=vj{05YEKaP3{cIQ9gO-_#V zGxAMOPro!je)ikS%t3p1uZ|~BMr_YD8lm5IVL~P3@JsCS-`{8CZB6*mDoosI2OStq zoe}LSz156a zz2~xO2EzQ{9KcApImTL$e&zXCK`dC_7SQXv(rg%~XJ#V3$~`TIvmzVu&w`t+e;-UL zu`8)*wY>!ve*CB01!K*hf8P{X87g*rd_I!kaeCrne!N9lI$32dDao=;6`Q44BrV8d zHYp!OoDOjdA><2W%Isb&XWPOY-ZCEflKYNFBgg8A-3}~k`*h{%bfMHBz>0b^EpEeB z^=i@K0TA~y&g!^F{s{jCCctL377=(fs~Qy7IIbfk0F4cYtBo#yV{^>OpuQ$bV_M{f zCt+=x<6!vrQU{zU2Av#(1I44}Ac80AgMiM?2BuTPKLdLgue*a?WWWbHMOr56$iK7$ zz};G&`e%KJxg!r$HJdnK|3UC_@8HcFd(tgWlwMLf29crZ2%@SuBovQbNjtVuLm8>E zpcd38`K{~XNI*~ij=~^Db5&*Ah7`<1>pofUa9_PFSOGy?D^`9P?6t>>dR|GaOHbXU z1c*N9M>T1HSl>WslpgZRe^k~(5uGGq5?6wd52XtJo(4)Ua7_Lmr1bGb6ph*N9ilai1G zM~28SXja=y#2f;3!t+8Ga*GSwQuoigquQ{81L2MW?~WABy(CUI0a{{OTJOS6@{-)0 z064yBWlesAS}Hi1aG-qus_blGbHeR~1L&#phogeTrBTi+_O8lwS!A~BPB{Vf$S<%ab>$q1iCPt9YYW_0%T|Rm{wJL^w`q-I_x>yg6!n%W^hQQTYQO zrSa?+cMQS+7X9r&MEjK+3N^Dt6djHBvQ~Ph_x(MwQb$A9t_7ooZa*E+#$*_WG%hzM@V{$j zXb^_~eN?BN*Dc)zG~zZdqy^f5xJ^lSF&+NYsb`M5vF7HzSBVbSuMXp1xg25&qq^vH zU#-r$#c*AbN+zSH@sVFbCAtJ&v9V!i_a(6ST+^TOr`jT(=iY%#X0a;jr)Ei@!|j5| zem!CL57)>hR{GFomHgu)g71P3^&w|#=m$Rjd(eQaAe}Msl~M8P?mzjY-%6WMZs3j@ zxY)5zG4inyc!haimyMnKSHj{q7hOfO|3r_K1Pm0tVUgwDX;s`>z@Ur`$;l8dup4;N zm3MHjO$s-6>T`Kce0=^OWx5DOBD=$psbXFrE8Ysc0v6n7J$Zm8TO-MVc-O8Kz^Lb= zQTR+VvpU?)W@dA=AwK^1PwxTFBAzLhmqZdex8zZJG`wd#UQE7C9o>w_NUd2X&^MGMGUL+f3yuExx*{D&;aD2;4n6bE*zb3unl4EV)*DiLa!*|7rOpnEh5#R3LyCN>_M(9>PsP=B-W!gXh-GhoTK^}@b<<<@ACe~CWN z9l3&E`uQLhqUo=n*xyieABIX% zp{3>8RCfNm&rb6TcOj|~-vI|uExF=1i@FX;oP^n6W*lP)A14AcCy^DZLYJvB%WMc% zCG}A`;B?mBzRxRo;f!K+;f&~y$>3u^i)|8JQ*FEY@hYDXz2$Y}H`mpsZPhZz24!?2 z?t|nc>A^H|yJ?wf+_TFU_3s^)Vaf|uT>!MatUH+28kmd2sEsfvK4`)SIXA^yG7v9v z*z!h3!wizA@E8A#&FNrWlIZJ>Gb<(;BTq&kkjv37CoNA7LxR=r*5gz$z+p zRZ^78g$tmxIKE2ggrkW_S}=|b25!BE|(?l_DldE3Jrer`u-jFsZz0q z*|_uVS-_*D<=d~@pFo|!`x})@Asn;X`{@%&&shwcUf#UuqCZ|Di5AGt z8B~WK*pM|C>kgx<6fG<$F8n+9oCt8w`8Mh5k0FuyiVDa{4gAz)C3?GG-2IfJ-2ddB z;~)J8_q<$4kC*kbz-=-K9WE>g!33czgd#Aq*|Z#SHPveK46oa5!W|9XtuX@M?K zsnBFAv%6~s6g#xcvTECfQ0h|2E+Sc>z3VLwn#+3L874m>pPb?0OMEMGHc^Sa*y`OJt z_V{8qLHS^F1O+|hphEsuk$SxVZG;S!&?>MkI5^fw%9+Rd8Rjst8UQc*eSuyzUfe&h z&NH@I!EW~8y-xL1r2Z*OWIV4B_1Re#it$*Px4V!?`&(FH=hW>7XTCO%r<=jf`1e_rt-er=FcaP#7N9)LLXOsU9 zV@q>+NOK3@IvPCLLA)_Wlsw_Vl#nStwqnu7%D^tL$*~xQZl$oOH3e~kFM@78~YOjBP$zx}CciK!u zjKhIzyVQ0^N8Yf%UK$y)WM&5FSQ3D}^FUGv3L4ZyBLCriC!a-;Hen#+iO!oAEN|7g@Z(=UEMd#~caUV5b^am2q z`47L$0We)$%8I{J*PiVZyIV(d+!c>(Z=;vm9VI2Imgy%sz1+XqTK)ETJrI~N23x$s zN)NWy{Rf^eYLCdPV?yM@GAb2<@%OKC8{JAC!318n4$U}gHu*(+_!ZCjrJ?@t4KWAT z9v6$?Z@+xMe9(>m|B4n^mQJ)^eFggUMss5#`R8cz=L26PV0rxo4I9~ie#9@I$VwRy zzBo7ftY`muasKr^s2Bhs{7%vQ|4wNl`%!>69bvWr6sbb|XMNvc?ZU1{-_!-oSEY;6 zUw>?CE7zZO&qgcMto5Ra%uAOkdcAvi_!VG!UIIhmPrf?C7E_fdE#1B9%`-fA4lnBO zM`Q&v>wnKDU~{CSQO*hEasZ55{!gHA@7(_s6jpI>Kn0R>1%c9VKECKfYg(RSE^1N1 z>?^6@97D=tR#WIE7aAUcIzwj9P>lU!c|SVE^7hb??BKV#J2KWwXd84)SK zCT(P%-#2?se{0w42E6wVCit%dU1r47Z4x94*Prn+D7jhmb0gi zZ$$oAI5<^`*27@W>3;p0{kYpopSG7eYqH8o?)weXQAbo@aro=n(Y+1#>&0SVWlXof zA@Pe+<<(q$NBlUN**K)FJwQOVMqf=b9^K0a9{=_mh;O1`zaw;T<$PzR)~s0h?{d?z>0<0Hv-JV%%#={?dS+upAd4X8ZCF5b zPkR=9W2*CLobkrole-TwzcyUBe9>PQ(5%6XOL6l&zdQIcg_cI8z<)SfMt3=H846m| zs?mzipBfX-wg@wwX*=lKM?cYnXrc`6G2TvJl$?TwM92&XHw+bZ@Cu1<&W zGwM_au9}UvdQ~T{uK-pX&57St(>J^&aO`>Wjg<9!)>1X+XIY?MzkQ1gtje9tk6@c0 zFCw@|vlMRT`REJ$WLCFgwZ?t5s`U?Q^H<3I6IAju@j-BUZUt{Rcq(i>Xm}mA+oxS33R-nW{)haef6F z+1;zr_O+B{;#OL_78ki6@Vx9!D_Z{=6=>e&(%rR8v)Ov4p!7Z9KUw3d`=mmbyU+E) zo`3vZTTFd|@C|qEweT1fnw0;wv?!;oOBO`Lar@)~xusa%f2?^YcxV}2H~kN35lz7S zz~!;-q7ZA8qu=_l*<)lZ27fx}j)U2HThN09E)Q&AVl|ehc9XaM+2hIt0VfvlM`r6# zqVf)9Kx+<WsJe8C^;9WHR5@+OzkBz$@lv+ic~G zAQD2@wm|-Hpmv-;p#1!x8x4EhmiH79htY?Ik@`M3a`oA?(pYBw*)=^3%(TPU4f2$7 zZ_8xTJ>@PVzTU=b1kHXxqwElaP`=6CcrQ4<>t13e{^uymSAEpi3VEV_#<8{_Kz*Ug@S;b!~8Dx-o)IE^e|h zUj2|)KIttk^0$Oo<`rh6E`K7>fxx=_@?LASMEPh%LAp4(fjtDgIyIBpgS8Lxju0twi-H9`L(u`_gVv@Sx%SxVntfn zc`}MDMiXB-Nsf0P!$@WNo>tp$4t(jxc~>Ou{v|BB{Dc7GMYp4~OFrdNX1r znkK8a$>Tq?%Axk8Kn?Ndul}i3Mt%81tE^LIacgu>ETq*g!0d`H@-rJ>(7eD_G%L&i zCpDb7DpdtM(C>xja(~lr`|8wPPB80D)8bS0Q>uhLPOl-eK-NKNba;4S60y&G`Iehz*;$4%gx5gIQ>-BU8u3um2tS5eE zE=)dK3#>KdFxpNfM>DEhAmUssRO_3VIC_yQE~j>peo&loxSOW}n0SHOMbB&YR(-wE zY*04X2a81oSNcR{jkKR(N4@r(Df7;L`hwg}aPtnWh>64eYu6#V_e2!|jNddH>gz~t z6JA*21K@IAtLu5^(}($^%My))*r@<~tI`Ssm;Ekn_I1ZQ>OXn-{n`v60l8SNRM_I! z!BMTY>f$6CZk%>W#I6>PO8CSmk=7rQqk$%9t~vai4ZMNPCAP5Np(Zet`K8K=Fl(i# z%ru`!`9Qe8vSxmBLV=Ul1cddCRRUY4W|H-_&DZ|h7xzd%)~FzOO&V|L52MGin5Ggt z6LPwyxW_yLm?F5Z+$bdJV-qYULv#{+N=mw*e zJT-^V-f@!iS9V}^f!Rn(tjyAISAXVxPY2onMB-PHu|1i`EH!wCY#;eyZOYk1UMaiB zOc>~9owM%sK~*k_QRT_p)ng+LlZG|qWm-Wn+mwU(u*X^4ciLx7nJKu`INuk>wcQJShi05_N%X+)#(UQ-C%vRGDVzz{e>l{Cy`Hq8V97i z;JAVO1CT-2zE`vQ1mdR5!2HiNk&5NYjIHZNs_@=#&3bk!%#HgG50Wk{6wE3xd}P+& zeSvZZEi%Mm`UO-n7LY;B#&Z{V%S&O}dPy`nZvMbFyto|ol^C@?dHoRV-7%V06MHs^ z>}xd|9eJY96m{%(6?h*B1AA-mbD(lmG3s~m@rKX3aoIbiB6ug7x+T!v9AKVaTde?N zxYPQpbY6@9Nb^Hy17|zPvfOaC(XK$e?y2cnl&OuORLLVPcwz0Pt^mFF# zS8^g-|4lD_-MS4PK;ckaL$c=3*m}L$r;sf}{@6f5*vn@R`Dtu=)wacO4dDt^>nhuF zgEGle>kQBaoycQMzI3J7+IXQK8iL~!!qX6Bh>JmpQ)N3%6 zAK}#DT0pLGT~A7(cHg|+h9`e$L8dSFPzcu!Zo!c<&su=OC)E4bRijr5IfQQ7LkJrh z5K=SBbzsSMRgKYKpbRz-<1|fiaot1P@HJi*P?@Z*b$PoVHD?#^NptXtin4WdQ_3KW zI+ap?`op2th*1JoEQZ?&8vCJ!R!Z)j=po8S2A?W?ju?oI<6yqc==U zK#0+#oE!XMnmIhmqd`@TB&3A^)Ygh-apBLmO`|emS%?-=cfBQ4@p5nsQfN>9>)Prw zqTddy#djrDOV^Zt00Bv)7Fm9)F^vX6w%{-PNv8-u^}v4_MO+TFZvW{L>ca{wgm$85 z%2kQV&9DPTp+13sGYSnIz;M`U(*;a5Yow}6um-799!e*27Gi}R>f;J1wYgp%u2di2flQ^-nj0&A`YqCX6)=h@wEN#1MT`i3 zcCZ$p=41y=_yMazgK`itaAP+(Tw>^kQ$$yZt)Z2sABtAWA6^lsU%RGw${lner;9Rk zW?B7wDk|+fkpp;XRR%WF2w`DQ#KVZv=|c85?sj~*ECT{zePqkG{KmfeEL3BQSpo1tsh1``qsw2?&JL+x;mrXhMTHm*yJkal6`6!W!vw0{PyuWgPM`qYs zXpVsI>1f{Q$accq+8`1`m%8FK@6JT+!-srG&49;hD7CF&+0OT_?F7|zZp7oA=>5&n z#XhjhMHOOVHs8^t(;?K_u*=Q;sdL|Gw9AFvVfVF1rljNSr^CP&;H2dugGX`M%$Bj9 z`MP_Xm;ZfXp1%a$N|Bo8qrZ}Y>pu)W$x%(u!N4~)9(A6RBN}hN*KW6@z9NyCLzO!|2m!DHrqH9mTUANM6(ZPY;s`iV<*F=i` zP3m6~`rj-uZXJ0}ESKsutF0!*?5Rbw)}O>gjKIQ>x0 z-jDMLzl?ro?y3-SXaun21qN)qg6pyQ)BA;Y#N!lTmtx5hk8` z^IDLF`aC#Z=?(!}ipM}8S5|*Qj>=(Q$`dM`LR~K)@RW%-zT({HH-iTeOgh;IXNNn@ zz-E~By1cR1;awQcoAwWdIhw^Z(#gD^BlDy5b{c-_U4Ze!D!-$X)4b~%M_~L5^!fS! ziQIK#YoAI;M|*P;juZ8p-o;rTd=E}QpeM1i%$sf`O)EpusOwL}az zJF~~-bA>D+vJPO6Zfq!PS&!-U#P@wEtCB=YDSX!G=l$G_?4Vh2zKYJvm* zQV17#^;ZrM@MxWc-y`CZ;-c5iDPQg3^B1O>Z}0!*r|+48P-?^w4%m^!efs8$sw#6V zahHpfloZk*fJ4iGrDU~#u&!XZH2=vuV%ZPb?(*=ge@OOjwf`sY?g69Zk|fYXESo{$ zwEUIDbd|O#kE%*ra9oa;6@AOxoQ}4<(;Jd!(~;x%2i6eY83;;c^%k zStXtNZUy4Kd$G1h+fx?x!Xd$S<-rOR{Y3E<5kj_KD$N5jm`^ zl56^tGpxTEhI2U(Ihia_nQ=rV0=wX1R*zP!x_!PB%fQ7mDmG^T{dd(aBjZ?C=)H1O=hhi-(Q@fkJeHuEhk=+9on35V)` z{4=oYMs$K-=r&EmR+{ss`@*MWiPn*;AffZM;{Clz~0Px$}0w~)jQKqPDh}=XE#0g zBIY7@*VU-konB6w1c(8YCUPlDU+RsQalbV9=z6|>Hf9SQa~D;wFxG<>Y*g4!QJBRM zz0}Cn#X32>LT|7uql_UZ|)OE!cp8WL!xsAwF4CD))##b=P0Ds{pO;Nspa%k8H z{~1alO6h6p#s_ZlI?Bi?%NefK?=Cs))J_+nq^OLfQkM6RrpnBka2rkF+zG-OB6t@BRndK zulv7zeOF?%W58_6DiW+&vJ|9bh(Al-jU8iRca+mq+F-=9Wd+7(&C5n>aOIe#VPR#oI@56)$@cWb0_6C zjh~xUL}UG-`|^bEg1X;*1Nu z#0=BO`}*H)uY&2e$I3$zIGrSea0@BPC8DCP9&MUbk8nf7TK5eZBNMumlaCyI8ECGa z#Wq?gd+j^^5~8<_G%$DW;Dk2COM4;pzw2 zpduOk$C$=EQI3?>M4yKVc^oo>BS^C1pM-FRv`VZ};+EGyAtAkfvV=ceFTIT&@A3r90sKj6D zUhiXQq0&8>S);wYo&PMoohP2kNRCwUehk|nd`zR|m^w|Q655nQq;v*EMR&s3apKBC z_>RRn?V-<$q+HaXV};pX-)bSZ`_blth*|SQuAr<+jmHZL%(_)N$(}IdsZY(P&EJw< z++G6$*u*Ot>Lqu_Fs;s}NG&*VT?k){dP4W~w8QS3jZwTX4QRQUEt&h+Dtl+Gk2N?? z^yjf9C9;Eh6pVpu-yf8JBMIgOf-`0&s;m1KLwocmgkSo_N{jVlVV%BK&I zA4PF6&Tq1-#g{EXOV5-r%Fy$ zCJ!e{>?2%{E>wWXoT@>LOucb78wDx61fJSPRz{EVBHbcscEYvI@2RyMMgUQSZhKLq zsiMP4l~-A$^jP`6FtN$7wsPZ97RL&-a(6dv%=LhE!~MK3@Y=gGrC_c~THDDfZY?F7 z=w%$e2HXDnJHqpxc=pEgl)iHFV(UG#d?yKF zw{EjB>eRtq=~~+;KwfGVOEb~FIyWA;{FLZtR_X6o^qCp*z#L|3qnHOXWUd}jcvu~L z{@2AG(&Cp1h#!(!f|?EK5)62JiB*5U4E<#0h)u69K+(XWbeO-cLrM|!UKNni?PoX1 z7Zx{Y`yW9q2tPZ?LuS@jS1FwY_hYGz@@KVc_Xp;WZzWs!1jD}YwEv!`17&4}j}$cK zS4LDl$Jhmf9XznWiJ0w3jYH&D-ud+MHF1qSrAV>z}UP zAXZ3}&+HO^Qm&jkOHp^8R_(yoxXI+YCR%=IgjAg&>=#9&G?}phGUx(FZoC|a^lxp| z9W{#&>O?`Jk;jzf%~I9J3`Z5tp+EE#m$1628TgWT-Ne1Tt`hH6ev0E&1BKC~Tqs^; zXPU>>8i4UlYV_K5V6%D0)IRpR%oG%31!8ek&v!miMfUu;lYUFU$K{$h0hQ#{)m26Uw#&ED zcx?I^YAT8{tOQtunO^4OZ=$C%6&T^TeEuVHL{UteJe>LI7MDgN_{qTfK*YO`B5~_R zwf3XUQyPFb5J7KQMIx7dG(}LH#4Pto4`+-sp%mex5rt7GCp2YCXn7z-TMzH#{1x-o z#tT+|<<(S?!OOWLoj87j=wP0a$(ZLuX-)(i5BdT78sG*l7;+1uoaqCmQ4>Zb`fjxI z{wdM$@Dx-k8geJ>oA3IqO?tJfVNP3a^0@|iIDO6-rp1-25p}jO61&;tPyukH@s8bI zc*Tl-P@?Wq_DxS~KLE&cO=mm`V{c1;xSFzk>Q@l!jGc+G2Gi|@xi_$yo`Tbe# z0mU{oPD-OmV?R_f{BniUi2MEU8femBuj93Fp!26l$-iE5b<+HIhexiEDgdkA_@=ou z@_qEDI|W{@oGICjvmTW0ODVkb0gU5uaw$^rWuY>cwZhvpE+JqlJU*yZUr*~8ckojl zx;shR^g8bF*z2I1m0q6DFp8vyQ-w-k?I&L&t*2sCwS#G8ANn@005R_Gl zXsQzQa>J;h+$PsPQDn(hlY{wFE4d-A|_us({oywk8atjM$Q~UXIHM*+aP+1(xWVEHu`K^BXyY##gS;hdE(#1<4H*dEU&;fi z0k*S4SRg;`bog?#c5NayU*z_no=6YoR|pt#In)nVPCdU^k-KlTF=F!Udi({iBj2ez zi{c=KJPDz|oUmHS7h4_4y~9>CQyAhh=(WAdksVww_Z2 zwJTId;w%pWXjl?(xYNcHS*wv1_^}!rR#T8WsTPWLrq`eEuhKs=>Q5F647qjE#G z3^K+^cx+ef!pRboXAe3QT;dClCTv$n{1neNx`CTF>~eFoIZ?0L*0*aNgECQD{5rJy zbLWHIGg`c7jN)04`ka>$)mW39z*}NjiS-%JwW?4iC-0J>CFm9rujJsK#_Jk#r8TZ6 zUPy@)xd{I~b(jdtT#^I+}gd+qg*#!me7zNvV{ztRi;{n0MMZG?2~0l#U4|H#h$ zvwMDp^Xvt5+V}q}pRuv5##bCGUo={p$6!%rW2uypx7<*pT9*}%i!w|47|wZ5lonL59%Gp zx-$IMKGvd1~O106aHQ z=}`o%67u_SUnErbsZD-)&L|4Jn)P=Yc+jy3>Fb@jp96Vj?Bg5~@tg(~*XK!v`Gu5? z?q(moF}#NI02#|+8+VXy!Zb*w^)BI2%_BeR%lnBc}{R2N9d z^!8ba*kA2eEu8{VfS0JI?^k3b{uE!b){|8Njo|9RTMuYe?VVL9n9qBxyZi#qF5lyF~JSXYv`YGn2FwW z`Ye-U=b?o3;DNYn%<9b>E^y^zRXE_Kai>he_ud z6eS}mK}j=5xqP-?5<^CAa5u9Ort?J zwP5b}BallXNVGGc1em@8FNae><+}1WfQ%Wobe=Zfnqd|_E5Ck8V{^Rw+6mN4Gg09d zz)@tAY09*g`_9zF|1F36Ld+XU{Par;S6(5fwFvvi5Q){Kl6Q-oC+FQV1>?2q?cuEw z5+U1e%d;-s-GFz|tJ=_SpUL&llCM*+y=@K;?~`-AUt@r1RAA{eZo*cRJvzW9Pmo~7 zg9YVfCMPkHvOtD4M#APogwZ{opMA6?53%i@e?~T|b58Do2h7Ce!Z5tPy1 zlr{7)F>dOql)J-F_pp9n-8u|jnM~;cBed~JLNPuJ6f!i4#4BoVl%Z8o+)2Itjhu{3 zHJFeuZ&omUbYuT3A<0*ZzjB-P?*irS0lUH}XJ%upeDSB%Gcwn>ihj$Mna6Wex1|zo zQLi?$KE!OiTwsyv?T812Z8L-OQTxgfD(U(etMIJ;&P;^3s9q7NMFJcgvm{ROQbbEQ zWet5iY8o`xH4%7yG?EkFXmd&$nl@tt^!Err5*N-v8RoeqV+5yw+Uaq2fwM7=gjGcs z+L(gemlzJ1&2#~J=ji<{FQwCd`FukyJos0~jq(bv;jErWKyU2qZ!`z%nvR77C}f;| zX;ys~Db*{{ipyd#@%)l(=_Cu6T>CjWqG~~#ZkPNqf7CQ3Djhj`Gr1OLDRJm40~j3! zrdH#n3u`!x0gv4|yi;VA59*Bso!fSY@_K{=l67}zc5-RdyV~@fm#R){KZ=%Vg)ibY zK#OgX;kzV4ix9;W`<8vZ-aY83skjVQXi%Mv(BaM;Es$0e+FZHsAsPwqi~~syr3uvJ^bn7q~wdK2_s=KFaozQ|jl-#~NyS}qt9J&~96Rv(m`AhrI*9Lw# zBnRxwI~&6qW-EhF(g8n1mJPcVg<-)&g=EAc_2t`(j-rq|P`a(5%!l$d`gr-(my6x- z7_*h?(lTIGiQ2XnqdZ$ka=3v;x1WA zE=wFh6yM&=c?k%N5PKnD5H6btBV$(a^x6oId6pUp-`do!>WRzIR@5>rc4yP=Z|LAXc%=@RBkvVF(*`<%G?9BL#j%z< zZzCAQK=>*n8J>_OrLKBJ&!t4m_v|%|>MPx)w-d$tpcCtF4lrm&tYMi+7|A?zF7v0> zniib}cvJk8T7l_gIxkBOFTJCSZB%<950^nHe(OnJEvW09qnkcDrrk2prB+ddN1cU@ zO`1ZQh?dLYqW?TzEhK8xv!^REFVBUSTF{5j01Bpb$U%lFv!CG5;buY410|E;d2fWZ zu?gSIRDwbRLvVj8t@%k1a@_^vll>iwmAUUI(DslLSKv(9_Z0dMp%sOKvg7;8=OpaP z3q3+|vR9R^d!yly4JXg+jO3`2_+2r2 z1$zz&E-S0XQ@h<67(e`eFMcYA%}VGTdutIPl!~4P>aSV*UMY@mTHgJ{;N!v8KG?(? zPB8wtl!cEKe{`^Z!c6$0T>y0XF}2>M54m0XsSIbLmU0fey(ASs7sWw`f4}j!g72nC z8 z6Spv{yWW-DWY=+{LJBEMz5Zz1#xM&^cy&(hT$X`a2u;bEJvgZ0e392(@^rX8aoJkw zg3AFKcs!VRsyWp=$_Xv0IkGfj7)_T1mL5S+`|}oSX4dspokCU<&STYv^9!4Kfpf*i zUu9vWM~}jR!lN^sT{+fciczhiNiw}wU_%WZuc|W2;A94b0tN@{03%#mOWu9ONnnR6+V)1I*dI?CB5C^3WuJ z)-Zs@a$52fzGGT4>^a>QL=^RKl~I&$RAg7Fxr~*#Ncr|y-%#ypgrVsox08UushREa zASznCnW41K+o)R{j>e71u49$T`O`dD8EtpU3;OeOTnaQ+Q^`CBYrAGi6?78mRYo4D zT{I6E`PS!bUj-uJ_RAm1N(T9Gr}65a!aj``I2B_szOZRQgx5^w@)|$ZC8tIwDnv+O zhwMaKOtHr|8SkZ$7qk~pmi{%n`S9Tma0}cc)*R)oa8o{wPm2l!`oA9*5xHpDPH#t; zmLUxr7?dHM(K=TucL_DR>o~f5h-K+K=xm}YR?;0Z-Di9#c60VW`}>0d!wjgFbrhRf zpBd9)xf7GS@*_8OWv*bZ=!~M1I*5Rc`M>1o9E0bVgro0aho>30r465p zCHPug_8n^FE?$?~z{n(`cOi2hl1>TtR06q~3x9$(H0?8T>-9~*+INt~dPgLb)}DOf z98l4?i(;ShMQ~La9m4zSq_oEiRFsM~lrEPYXA6`CuXUL+iQeo*-khcT~7k8Q)& zh0EEE=;FCVeIl=|IAl+~ClG(+s&G8r$QPC9xi>PlvRRr>tcj;eUG;ZFZXnZ%j4>#l z#QULltXYvxW8oOC1xOgXq24uUL~i&P+GL#FUYjdddevde$ZZj|=jZ`9lCL}zT&dff zDDB3$TBo2K?U>)zIu&1)o{`cnbayG4d&Z1BiteXfy+Df9;i7msyBCe*-*9wEqgL%4 zyQES&U7&iHlt1Bo;pBX3c;P;1w%cIHJX6MNs7542nxHtWxj{eB>C%zV#D1fZG)Mdy z>mDQ(hV=gWW;DV)J1{4?TOm3c!GaQ4VKGi7W^h3K0h=x%eT*47;+i5CbpU4(oUEtH zvE!(Ny+G@UPHbrL&kh-Ff4&E&~JX@&qpA zNc)@fH1#~2N+=QbZE+yd&(b!P1_mu-4Vo|*%; z;a2yv85N40+v^p2_eB&9yAC3YxX#+R&hf@nDX9c5r5-SP-axk`~vq;Gsh zD7`7o)K0@yS76^$cIPy^%}HU)JnE07Mn{z^%SDGuo1^Q>joN~&SQf7Q@-o(3hUX7h zyGjG7Wjub!7oD6oi{xjPFx|zE(rdfX=$8I^J9w~`>C1Ws|f1cYI2so`o6)MQD{XNqs2n=ue1I2RyW`G=P zsH5NIq%Yv)%%{?FW=*Ui5^o^aM$1jo8Im&%6}*OlVz_(UTu#`Yks_pf0nQPv_``5| z&VGS7YVQS^_^2D%0iA3`Zgs}ylNWt9)KSNZuVvB@JHcYA zzM9(D1w%ushP6r|yIv(8(;7UAE|Pu~IFf$v)XU#7^c#`39Kp%8NBqJZx_Jc@n$Xw< z*oO|@1g{++?QaYWqcE*f1x}LB)jHOQG|5sk6mD8_b*@i`ull^v+C*5QwWq?UkCVRT zrr9FG#&j>14`Llsg^R`>gC8Q^MQDWP!hHuHqEW```?!+RkZgl-}wT8)?hzuF^Q~B~b^nu&{)mXW+kAl+WSfoO|g> ziEYNn^HACN`>)N`AGGy*Smh1ne7O;%2?Z@UZQW=kllAQZvROrwr0GbgJ^rD8aJ6dsV@ zU*Uo)_^)U~BjW%B#VsFVr(Np0It)q>g|eEnheQws&nbn{R1B8q2UvwFOFwpb7u}=u zk;`0XS9-8@Fs{EA&YnG#s;hl+!V4shK23aKDZ0d*tsOYKV^(`GWqHIk>QYbGC6upi zSyo}e?!o3pIHwI3tll_cS`XeS^z0fa$r@TY*xC`v&5onM69{&vIzU}c4D16&MI8Xf zWOYVSzdIdL3fA^>1ehl?uYDy*=gUd~JwCwAqwoV;q^}1t> zo+zKN&TQbg?I(!xd!3=~0o2NxBTat5lXa$YM5o9>)5E~|q=KjgJg`6NV@)}O)_%QE zyfa}4Ro}lInm3Rx2T)8O5Tv`uq7+qPZd&%wcbeEqueSCz3QXe0*Jt%pk!j;VL@|&v zAco|Jz7)?M_1RGc>ZmiluMXKR;%<^bvus-`kGdDWmC;b!wt-ju)^RkDq1T1gh(&Bs zn__BRv4ThnVvF<9NSz^6xPQyAEG(OTS?oCUTZIl(rx>#Ssop1e>Cw2Kje8t2CZh`feoe6@2OUCtM zTIl74u5}x6JaKfnNvm2*SI|?Rq4u)J5aFY(BV#Uv(lL8Vn{UW`+)RSFp&XA* z-cswCBqt}gMnR+x*9#Lx2lwh6oq+Sf1)sTY1RARzczt>9NrtkOM$r9m)r10}{tIxH zg~w;`Zhuo$dgDSDRYKK0Z{E>cM|jAeqm)NXX_k+`P@l0tDgIF-dt9mgxeWC#NK3h6 zXRC=#W5@s~u@utypLd(s9sR{;5;PTYZ$&&4Mz24RpZQYq^!=NdnMg5^e#%IPtlv1n z@#zaNfj;ez&*w;lwU35r_0l<`$FtM%1%Up|FX$7L7L}*DVdoqj!QmgfVg=>8!BU8s zmkz?os{m>nP3!K)2#{+4_LF(m=}Ku(@s_9dp**uusG3?G4&qaMSAp%SY0@{^Lldm< zy)oyh{lVgAQq>zQj+F>SE3r69_nagd$9{-ze|iIMdYb*gg9jH7ijGFNE(=ziXlM{{ z-N=B<+AIpqb`Bu>FvuE@{sO0R=LAiwy%3wl5Xg0}Lq0{}R}CDt3QT*6dt0AF|5P4e zyYO;hnxpMbYHEL%*-_9b*ElJBeZZEq<)g96dM{R@9ZXEUOhiMDm|v<+d8y|&G=h(O zJwBa#voP&Rs;ihtue;Dbms7M?aldCMR%&&|c{8WqJV?gCf!)ht^#`6f&V3&!Tzm<` z!C&H_{SzU_BaO-Z6Pu2k&nLxx+T?$H;pbPA+ysOQs7%&svY(9O$D{t!D~B6$_Xn6r zNq+wJzd^|VdqugwchG-(^*@+L-*G4AQR1K0Fy-IuPP8G5`?QI6J%wHkpWizeaj8oas2g=K2 z?Oss5w#sGWP7k0PLG&#UY4qgee?eU4E4(QnP5QkV;Ipwj$zJWxLlR&D82 zsd%}iYj$G2R=IhB#=vp9`TDIZ(#P@M05=jOQNDPc=p0oPx3O{YTh(4Vdv11VS8#4LjIKs?F%H&*<*hx~oo5iR zG`jm(%18*x4KtEqBaaD{o*-n;URSQPlL4+TQth#!!09~Y^o!*(nthLD4^`3{kiN2) zuJR}NVq%{DUB8YXBC_f+VZjqH7O+qjOV!wl@%LcPPrDP^xb9Kv#LOdVTZ5fRuL$v2qfP=MX=D{K=K)W@|rJNwPp}(l0gha$` z7+Dczi0(GP1cmWdtVp&qyDQmyzz2AEY|*oly|Td$xz|u~-eXT3N?j?rM-QALaN)?P{^3LOE{^{hL~7CN z|5>ZC08PXKaXowOz#*=$R3gF<*qL2>*ezSXX5YnffL88N&J^0Wa~SZc4enASWfpaV zLh5cxyr_Xs&{H$;3*3_4w812C=1}uHmLKfyh^%dFXrII=%q6u9yWO*FF`~QDY04xF z9ngfJS=iVVWK8#8Iqi4QsA)u$bVUnfuEuSLyT(LJj;=T9Luco~qDnhFKCIS7Nu?G; z-p+FX4NNKMGOH!P!Hp6H$w070(WA~lONr5A-=m;%8;pwWKu+fHNB!%~Ab$Fq3#1Fe zaa3I+IiAh&n(qNld7RfO2k$cd@(oG%m53VGYpp|N)@t?PRjZu9slfuwP^|YsN=+bC zq{8CL-zx<8ofV<)1qU2F>$o=Moy}gN_VvI0haAm0p<)@ib;7{R8`xldjaL1{D=c22 z7h+heV(&3aZ{;*mbDqhm@_={pf#ViG47yY!icKn|DrD@(DwU07X&49xh*wdyNoV=S zqyAq|yRBheqy)3COSTNw&AB41reFm%f>hnQfd(38Bzn@xI-20iC0TQ((0-S%CEmOH zvmc&)%k>S;^(xAFobupVqeahn$K|V6S;U=YqFCVt0_d+iKVUoQMMz-!>vY{W6$vKw zv~WtA1aNnN86h;gQEJID%Iotz zym)3$l^&1a$#$M(&(@&mHEp^cBPOtGUtaUFFTs z2EUkgly|ae*E9XJRmx>id}q+8_-Q{qvdZ%AU^C_*NN>CB_@i1cXLUsha!+R>ToNnh zboJ}NJv_v=iuH+}iD@QHL{=!xbH^iDy0-o49}J~?F$jLpcIP_<6cKq=CMs@%#?4ZN z7;p4$mlEy=Z74;2nF-=@O`_m8<B=ww*uQculf+vYTA%#e8LmBhQ<)aG$wrhXM z9e3xj6pUNvw@bDb)>pJsE|@I}gJr+jRFEvChC0|S_{?m~>$z?WW->;Z2c;cfUop#Lm!oKyjPi6KK)x2sp{-ZcVp1hjRkblr z%q8zrMPU2@pNk;fh5p>t7fSTCFQDybD>Pbo=&t>(@)CzB4F6 zk>vSVy<>0Ih{@Gx0S@I&Lx~O4@pcw)5Vo32=xe@+WY(HJqwfZLr2ZW-s z4x<&f3`+>BG}fo9LuIabqyuCLWeN8M9(E8NqR{qZwwbFJ$eXwoRDLy zwe9p4jq3Y@X9Ee@$>z0O4zt3Gh~>-BxJ5FT5-(y zz^K$x6%h-Z=*P|AyO~mV*zJ^?lSRgpiwPez*3pSU#_5N551t$^vo)FoHs477SN z+UVD(;zlCUM3QwQc(czs{1--<>Zq`yumM^Us@>l#D&0{Eidwr>1#rHNdnBDU-<*F` zQ{}xoXw8-*3hx#Wq+I+jF%n$kienADuq^sBGqNRhu=Sqmk>_zV2&)7%o9jZEG`OkD z?9KR;xy^Y}*1^ULDrj(f1teF^beN^)e_=*O;x#$|DWP*^Z106x_-4Y{6UDm@6(nqn z{!{qOVC{lwW_}Gj-GON?L6j$2dwe0+amz*Y3Q7l;L{;W_jE9Ew$EvF{oC9~ zaHdmcp4DcNj2r8EDaqG`!}xfp)6Caf92pqaLG7glb*wHZb7Z9?tiRrroiZadQg^k0 z4+8etua>}Qj+<`f8p`T~6Z+21;skt`xY$gIRoqv1TO`h6Xv;U$iQ|TTUWpSY^S?`5 zHbo2J#GqA+b6v)*km9=g1`mYgF+?EjlY>i2?g4MEKX96QG_3Xtu@ajtmw#f}CbP9v zviw0DOaj^jxx*zNH};AY-GJM4DU`S|-=1}axj~DRQsf`5t4r z?iAT*mO#59Zx$g?4ssnU!Vho(7injU6;n^=FqWZ8VI9tu#fnpN?bM9(yeX;t5R{F>9BY$!W1WIyv{5glR-RH(c3r9HVauW2i_;1;WRmpiusFr9$GN$kFOMvztND z8mwR760=PF)F*xYSWnH!x0P315w;Sjx^2u3G+1aWFGC1roU6KJTsrG0*~g@_Nr;>VCdV^Yg#+Fnl;<@zfvX8=vk8(DKn(pzipEaw@rnB*fFY)=B@z zgK~y(@-8Efh~8#QlOQf5ujmf<*=WB_NR#L$FSw*@B+B%;U~j?BBl8g6P^k2 zJP~{I2P69TA^!yrFglZ5yxL{5d<*vP|NqnKoC3faX@8ZtKMgaYn(#o%G&u#Ic`K-s zQ*10bKbaYQ=B)MaOi1x6^z>M!ZgAz)Ji@f{csEcDN%f$s(Qr7JnXUDZl2h5vF2sAg zDSQ2W;Hp8fb4;8l$c$RjDvE;S96}Yr+fhe^R!8pLw#HY4xMO~X8UOW;S{+V?hhHr; zsZJmye%|`(4sjtEYi{0zPoZg!^9YHF+vW-zRB>9CmJ2i0r!zb&68h7P!9gJ*F~DK! zDuA}pq7~i4nkZt|dQPiWPkN{p9)4F|QMH&Jqy!wW-UU=0vow(0fG9YU+t@F96ey7J z9|;2tqHtjbh%BYRW7c-OVZx_}U( zLcn~%yDgf1AnGIYFNWp+j_$_f+NE?RrfSR8qcH)QhZY3mBHSZE|Dj7>pdaUrJPcIH z2S5#goI(#BxfS{?BsSp5egc&0KiTCzIw|WXdgKR_9e~(`Cr2V=E8$d%5WhosA_b?; zU3~s|(@X@kQoSgHlyOZFvj3!3zyBE;2ss*Z7ArXJN))|3$_4su%B7h3fO5RkB8GeD zD_Y_^jzK4_jF0pW`Bnz=`{o+oy4ntkGBo=*$%*#pJT>{vTm?}6Es7zhiHTGnMLsNb z`K;n`G{hgpE0lT*pcn7}a$X_8Pux#DYqn7MsPFRSyl!#7qY#gxhGp3|`cAWF`FfHw zXFHLq+e>2*)Cy3AkCFwS+!)m~0#al3y{xW=gceh;qsYWOZPPjfWyiR=f+IHOU$Q0O z*?XjuW;;vF!pEm{oh?jH^>t^Aes=v$ai?#pz}XHNhS3-dS(!$%jAs4O+&PkS=L8v= zw*dc5`*U6lP^!^qA4&0fO0!{Hqcx4k9=BJ~<&tK73~iT6VBqScLy$gaIy>ZkXh}*#y)^c*<~*S@30< z&IFz%ieJQpXDm2QhM!j0m?4QZ8{2P-nmxw~u!)vNx5D4!v$Z4%AFYFfgU*4Pd7h-` zJY9M|NZE+AYM1Z<(HxmgJ*;BHvV1(4>9CR`9e$oLm&4$%PJ zxzL8F@<-a^I)}8UZmMI)G!F|Y-wctQC5gN*OcTewappT;ZZ7wdu%n{Cn~Ng1v+12{ zRz=$m*&^kyByEw42?Ld%V_RmVvTf?7HKKxUY?+;L}fR^VGds_)g01Rlk(Ag&NsMp!*u$#Aj&jgUvv*7vIukI+c zM`dng7^35BUydpTX`Z}?8lU&si9-NMDgx!@uF65lGFFH=8b~ZF2O8(84hgt-oEl^h zf<6Tf1ij-sLDArrlP*N}2IaP#C*=1w^I*N>G;U;`L%Vw~bWx2DtC^5W6S|Od#P;Ow z#3sYJ-6s#O$EbbX{J2JrOrz3asvsygw%O!KxxVsyaM#`-wDie~?az90 zmT{XO>ShK+kr?0YlDcZAl4|>2{|B6mLq?^!9uWBGTP8s(6sN5Hx9$RE39shc9EPRk z;=G)kTF+%9f$>e!BX%drNs*XTWIq0F+ZKTHH2&sezQ@>AEQqsxfN3j z_Sryo8uOuNKHu!pJnVYr-LXmVK>ov1Spw@lE0xN}<>sY#kw=8Xd}vY4Qpya{Xx8>k z_TJ~n`k)*}0z)lE_mz9%K2sU!n(WGkT*1_xJuMEcjGXffYvoB&j-xSGuDP4~$W=w$ z^qq$7nvYOzJQ#gJqRj%FiP4`6HO_Xu&cBG)k1Uq!+Co)~v-S8EO+d!h$(1VV>X7+? zFP4Q+$YQ~u*#K2Q0m~#HIO#hCJdG`f0Z7vepuu!#FQy;4zzq!IX(io0dUlqQU%kpG z?o2q0>NYeAAb7?qu~$m7Mh$Cg-}`b?ofrg45_?6E>xkLlmJGv-3!J#ak=p*?PGp&> zmJ!!5f96Bvs`oJ8MuoBp^zOARotqoUpleq3BCizAJbL2KMK?HbT49ybIlMEeo{t;fE$`I9(R$vTi>F|Dg*JV$N=6|nPn>T2{PI#n^WtS2XfagF{!D-x zGyWztEu|%5i(sz+v5JH#G1D@UylcV*yWw;8YzVixkZBOBq^h$HnQ2Noz>LRYOMV8p z@Ge{7eowt=|6ZTM1wpLfNp1tNmd}BXbY@CFFs|=GmB0SkZ;xh89kqWCX>Od9>1^-# z5&L|9_3k=QgS!6yw>|hrboSp@j|xu&Xcoha?9czY2mE$9;O$KyqT;?R^@Fkf?_r%k z_Uy&0TfEq&7riWf5CF{_KXz^YC7O91IF=R~kM6h(fW?iauJwv6y9|ALj* z{{DW4>NGfZa*}bVZp)p+G_L??JBaNXefyiAWaK9LQkI#?=JBgnA7^^^LQ{041c@AWD@$1*lWIQ|JQ(1nGONI+qQv-aB1rnY|%xZ83zLN?A>-?4}r}-rdufudJU&p{olE$8n?G?gu@+F81?; zf?0vCjm7yuVU%=jYZt*A=;!7v-?{K0S)Q&xNpbHt@pANPeeVPMGql7In> zYN`$ihvv&Xc>H=VkDxj@(f>ialf&v#2&X=FTjS=wM%{df9kg!OV`?1K9Ag@*SzH#v ziP&-3t1lMlm$Y0Z>{b8*XI(~PoAKLKNn_qX*yn`nC$i2Sjf5oAui^;^v1UH>2%y!F zGPWDlsb3q$&B``sbU8XL;v$Ka5ajX|pZiXYRZ+CE(fbBb<`CDkcbQ*V3_VJvObt8{ zxaHXg2{Wuo3#V|039m@C$1L0gmA+5?$^aG@EW260zvtxkRmCFau37!k5=H+-xycI@ zt;v!)8mVHq&2H(TiHx4P7#a1;*G2^y)z_KG{}af6^$lQf=XO|R!@+Bsn&rIsnRYo6>TgL&`vyQG+gY%N1KNc?x$bwd}=V9@C zB4vm>o>l2ph?d2H@%#{Ty!|W^0G)%Pa$1$LuRJPkP)_aQ@JV2?A1^aIJIl=+AnvSc zQ*_nC@326qO|)7~AkW0KSh8eEaxF!$v~X>1UfD_Z6#BY!x|ub#F3&1%(+6ys;ciqX z+j{CGVd?YltKt4ZrzXt8L<(r8jVEC_Eq%IlrFgQu^ZG?S7PW5~T2>I!oyA*{HE@Js zQ3k1?s_I)X*>JB44=jIxxiWbHcu*4oKj;BpT!&v=g^hf4S1@L$S{t7Su3uTW zoZQ!4Ni&WD8GE_!JGq~+&;>Nth843p?s-H88^VK6k=IWR^J@#;$>Y3rE$bz(wu#y` ztCv|$BtTyAWKuBAU|8Z*uV9YSu*8!|bhW{-c=CI}H)6ma*EncS z64CoMP-(u(gM!P@llEFaS3x_DL0Mq@C$n=-s_y7Ki{;5@?XLdfL2!P6brdO4W z6Z6XR&PP6_)qMcYrM$mQ7d^UJ+;DUqe$!XFZS6PlwJ@%SjDdT$VKXUUZnUk`}1ABRvfm*S z9ECE_NRfanOcucGxNu8SzA3{hLlZj|*A^}>AJ(gxkPq{foayH=rgo zH*rzXpko(MMMXXNo)2y!rCVjw!Wb>XT_WH4E zH$ON7yHyOZKA^TJc*2>^lFj_`p}?jiZ?O98!A>z*0cWj&qeGW!hKvg|OfTi1qdGrT9FVPBSZGGXeF zJI18j-{;lj5`JE-8`8g^tNRIX?qTdBMiWGb^_$R+8I^VXw5n;y9`J@OJzze`~%pL}d?ZU;o4>KfS6WR$mFY(x4a1VeXETG+XF$UJy}Y%lLOIw^Vk7lbHZ23FO<-clL}9E$ zI$KAAZ?Zo&HMPwI31%7%E|fo5NVwTHm%2t$^?&Re`E zAS#fJ_F!Ox#8J?FwYVm+@6*=PPa#as{$1Y6a{2E`W&O}SnVn(>-SeRyeH-+0=lhI4 zO%4EkpHaa#Ygz{7f6eG#0*BoV4aE}yLge>D`-IY2m_cX0L?Zm`CX(zhp5cyA978$xeYL%hmX}{X6-Do0j-QE zrI(vCI}qxh%cmP)5yR4;$_GaWL~3$6#PVC)03Cw`3@G*+iOV0@8m~+&YL7hMpz4fK zl$5gT;sao%yJkUt#p3(o%GZqAv{2J+A}-teQ6g%t=*W{m%cK-$B%4Bkbt_b3R%nQSz4HUNS|Gdvk!KpBRG?x=l{;e zmZ(~$k!V-Vbgx^;kLuPiu*{fp<($eK{_C=0``Rd+wTS^z}@YyLGxG_^`fnFsx{xd8{(a z@5@X3Iq{ZQw=6QG$1s<$uwzxBDR*>zR=!qJ;0Xc;}I6vsOxE zk9{C_%*1x4%xUXvt;c_0ahB8s5q&sbvO`ZsCyuA0L_FR-moUF)jO=%B1H|N*`K7mP z5eaT~{W(Yuiw_;?766UcqRCj5m67q@ODgWTG>2jSlZpwl%Ien^+NE|rp!-qoUbw-k zXp?GhWgauB)Qqt52bZISq$gFbE$j?_M3_ZCA>LO0n4yVTPp$~vVyT$^Yj3Nr0>RB})Eu5;2|&;G1cgsHd{l} z<+b_c(l9?qMN3#l7O89IW+2(ZKpy*ai$=_7^+x=ODw%FCO|FcdI1`{n(bZy*@r^6I z$%rd07h+AsPjw;itxWK)A`7&S)6zme@3vLGCV^)9UM^;=AZR;0QsMF2b7>>7OxYYL z$J+@SuoD(xSm8!CU@1<`snI5q=)lfzA}aYbz|18N2GnPLf5P5H6SOIYNf}CbFi*aI zS7#l^tw%9G0_~h0W_-Wj>@&5QNvbqQJv^g+?nA-lMR_ zy)BwVvd3S@p43$>2tuEOy{q-+c#|x%1}k=)qeRNJGKa#lhkGj@S2pkhdpPx9d-y1& z!_LkX9|`v+!4(M*etJTnAZu2y*M+Vgn`Ftk*?tMN%W4b)ha;R7te$y=utj+=TJMPu z;l0mnhHDbo;-UQrr7-8LJ;3lCs{MTn6N^NTl?tAmzSHCBQ)(n!ZNU5*nz9(6l8 z4TuD7IJl&AC#Gh1KOIw51wO&^*C%FZ2v?*6{!6KVCmm&Jr$|chudUL%^Ie{Qzsl=Q zA?>F=<)%_}ch3h{{qfD1$_kXEauV?>)Juih>!#y1@%+&AZPlc}^bO#EPo`uNFA;7> z$!cw*DJdJMJ4N;B07>n4u}4|V)CB2a7Ghz)2JMrT0AR06kZ!->#Pkc2R!)NRC%ZVg<)wcs;QwRh-ob!G)gJBmtHJ%#rp{PU;xpRJ%pCS#O!Hqp`4%67 z!z6g_{>#ArhPQ!%1QY-T9HIPjimg9=@?;g!@66F36aMq72wlMZtYYI{r~Lfae;Fg= zWF=sKRk*ho|73opFHZ#Y74&NjkpJnEqf|imG!eQhe<~lz>0JU?L%K#UJ&$C)c1|+L zBV`;80!~~fVRE@FL2#Xu(bIE2=p_-wO~#iu(q0q1B=Ul1uU}~j%X!hx@laB_B+Z~C z8I^YZ2I|Z!CH5G(sADO(H-c$M-A%MMF?ncWksUddySks;)G66hw;KGbi}~kNXr8-x zH8dfN?vIATh1Z5TWw!ot{qF+nFJvp>_QzE%J_D4|+GDS${%Nr;R>)Q4?1VV9bSKxg z4q1~%H-{#+hS5n{tc*^~G$GhK4m_N%OI=R<(aq;?fArg>BS~~}jf)*MEo$%9Z*gQN zab7WV@W`b*clPW|3cq`+A_EF+Ybtb3L)IrEseebv@U26DiMtPOL*7|!`;Xpq_1s4k z*O)YPnPWcHy(H48NFcrWf0xr|gTG7jeJi)xXxg~Wnn_dj+P8z8x=#@GEyTj+&n-!z6rJh;Ov(81f_ z{j|VvKO5Nq?=*mX$|$dq|! zbmNW7CohIFaI%E!?2xrtTwgkXx|LZ=yzk!=aQSjZJgwuF247{uO%oxoLTJqpp|xwE zVzUef5{X=&ODtAi9rwx5NWN)!IIrR{8PL~XV5my%uy-Gr?xKTljaBQb*6gkQtg2le z-=6)+CdV(n@Y1h=Ky+u9b5oTD^K{iuIB$*}M=|&v_=?hz>eWlT)(p)J>Vu-O`-+#OqbUX zE2%wCC(=r&wDT$bIj#F>kW)KHizK(|gvfq@t5MSS7ms-Fqg{5V4(@cIQN*bn(Fv!D z-9CTot^}`kqQ+q3+=S#La6%xOIEf{iO&Uf8x#%`lQ{i;9dule)ku2V_0*Z~xQcoN< zjsKGy11OExFNFARY#{`^ws{2>!o1WS22QI~6oAwovDieit6jBjrW#yv79K1#26c;j z#n;Kqo&tnVD8uc&YsK-9>1K7coti-U;V$o#-9=uj8mB=O!9o8yQqcodTWk@tbt-y- zP-)`PhHe?pr{eA9;99Kdj7w_b=kgnask(l2`Q|Mewae?nF$nd1j5WKorBU!gI|7_? zGZ=9*3Jdm=7gEZ{yH2hsm(Li4UPhzgXZs^$}z68BlTG5?3Az?8}I&>jx}>G+ih#(uwO&Q>Wa?z%F92q zGQIV=`<+OFwYqNJ-6krr3xdD7u7A9ZFbJ^iOVK#>YS^bJtMu{fVat`0md(pBZqG4S z1QLUA7c+mJG7R@W&)^$`>#y_(jw>A?@$BB)vc`>O<|pw_+L50atM{q(-ao|7(T;3v z%+_u6#2oIdUO(L1LKG1nsBe#!#hn+oetXk^4^+RmaIVa>4LlrM90sp(!Cgj*Z)A_@ z@tV~+RP}iOys!VOH}gAtm+q==SSi8lI4E%W?)apx&K{;fT28Q^Deb$_^f`Jr^4yB~ z6sx05M2cX5CY`7bI$YM*I%{vSM3G&kBd&0^tS7qzRyB4%t^s)bb&AdIT#ziNB?+OT zq^epk&@1%3-vFHWsnYm7G^xm4Fxm0;YV^SYU4M=yIW_;37INM53(uod#P?7C>+pUG zx1@OQy?SROmYFI7%S-YbKVxmzbG~{bj33Yhq)<>$DZ#7@K ze-6~IU)*^wa1QWb{(>5DhGo9|<}WaLvGiwN@)*`B*#qaXj}=&TejE43oLJ}t`;in} zliZfsoos|?eJMW+>nn{Ff{))8vHQfxTldW_t=4o|!@cR{tQ7-{by!T&SmyYEJUjEt zsZt-WdXncN5G^S;oId!lSm$2(n>RNcWF$ARZ(eUtgVFsXv7h|I|6#GeMIRA&33L}Q z(4^iQV04yxGA6yka=gyt^(}N1E{HuP7E`z#1(Mm7TcLG@whhiX=j#<(9`3nz_w1!D zmX$eYJVz(EtVPM!7Q#QhdF8yK25ibx#*8cGAwWe*%)BZD6B~}2^9kh}pP5VkWJ~`D zxA7tI4Shz4Q3!qC@(uB{#aruh0~`W=<4I0}-KYOFVg7P&36#mNl8QFk0jSd-K7E(- zo^U_iC;C6x`^B#b_*Bf4fBlcK;w5~n7#~9SuebG+3wQ^`$BJ%Et-OC6esd-V{=Dgmq z*M9D<(Zu=FdYSg9X8HGepDc}(3(Zh9_!#Q!xpPYovwrzFV5~8hbBu)5Z84+TZcCpu z?HywFtT<$-e%p$lUVM>JPd#-}%bxrmj6>y~5CucWZ;8n$!QSmc3{wSD)prm%PkxBl*Wu!e|Sm(3(IXA+ixEEB1$e@0jOohMw)q5JsD^#FptCL z8~atwt{tWA_U_$(Bk?L_WKI?9#ZQ!vT`_b9X^*pomZ1z-*8f^oxqAV&w-BZON#w)N zB#Bx)P5M+DPF^s&#SUu@S)I)qDGKu-S*c%qpgn|I4qKe9U2PC)!7}0w&6|~Pz1I-0 z28YL|MxXlimUC}7OuJTz^QKis>mTjEKG+{(Cm_7jP0!Op>yvb_iQaLVE-WeDA=1!! zm<66d&z5WV2&o=#v~R3;I)hd!o0hA_%-8eJ^@#7d>^d(mjA7XKi3_n*l$32jKH(QE zw!WpXw^iSH-cdU12k0uTiQ)wRYQ^Y}AhO|Kn9Wxl2ka?;;)+;_I~eD+o{LW(@rNJS z$myJlF|DD!fiMGb_r-hOzdXAq0v?^A_0j1WyO7V#1$4%&9TAZE&qF5Xbau(sEIMhI z`9FfHrxaovA!S`PSIoTiQv7h#f_{6DCl+7dInM$q$ZdLh_u^6ap7c_uStkycaqqG0 z#`;qGF(Yc(2L+wUJGDB633CMu>`+WWoaAcF(30$z5a-#nCl}(ZQ+%|14{)^+^E}Wd zenmQV>F{N4T!n4hJlLBPUbh7vN~&sG>Mxl-5&*uEQG%xIOA>ErTZGqLZ|Fi&ij6ig z7e(+SUx=yWPE5YPdOWiVgo&&5J8^XzE5=sWyq0QQ7u%9N8rpzf?A*J%y8(yDUsH1} z^)&Oq!eQfU2jd_LF$Dqq?wM|O6R-gSC)EisUWw8*3At>(__R z-P$n5+$KG-BVOI$;=C}DxFOOJ$eL%6f9VgXn0*xk3 z_1p~p3N7SxfL$;Bp*j2a$Ge>>M=LY$Q;AiUc)*>+qm6}gTKRgOj0SJr_b#9$07L^^4IMa@#m-cy2uWV; zD*<&3&6D!IZ_H+|WAR$jyyLbq3L$uAG~fTqKql#^$i;QRh9<&pqUcF(BgA1ND!3>H zpa!%ez<%^G0&vxojmvg5=jmO{CW%JUYZ#!FW0kP{0hh;tBC)!Gi~ZI4xn8!1+8c%Q zm@78nUoc}Fs%y-~g&VFb-f8D$vQx?JI%6Ekq%9X}x4&I6ee|r{wi+R|{qfF}OiIxN zkcw$%+d;eg(#-CVr!9?CH4Jp+E!KG(yG>&08)i0-ycnYAx{O|6tOxHcrYq9>hM4$m zPzo#^sLo`N@10)wAmR7z!I)*Qcy@fF)b2MHa=t+_&rk_>iFY^QGo4U>kIfun^|XPT zF21Bk<^YNo!XeGer=AcD?WYrwr&d( zY@zPiXvO0DQUE{k*s4Gzy|V-D_Tl#QxfQ+dSW&5n|*5UK@@CNs?M82P!)0`yqW_4KLI5ktq!9e8e;;rdkGYol_)0qF| zT$xe$-WU$4cq+KDyFp1}bz+d@qJ1>j@6d2BY4{HPwd1s*t<9q+qk$3g;71C3$-Z@+ zE|utk=}%_0h3bv1X zx?UnR2YKf#-@S|Oy2nq`+S#7R&dgiiPbiO==ZOxm-Xe{W2X_uiWUF5jDL{;@yfsq@ ztv93+FBA~oa&1-|j4Sgjc>I_EZHuvISkc)?yQMh>UMxd|&@OzMOQcHk3~|%&cb{$! zeZ4gpy}!k`VQ6SLN3-c_gtX<-8av8^@A4-Kgx86xV4pwhcCb8 za6_qu~;fZdDPdH!g{#cQ)1O^P0w6C+RY{21~q?>%}gT!w4-Fp7;dVL0x! zhlcyz$`Q8Z6@4sZxCbJfcT0zK-B$m7f}NuJzQowZ!qxm=YZ>q01T6U&&R%gFQ#6YRAdjhPOj?pgy0P8@bUHD!|6=3u_+0IWQnGc zc+zAr*n1a=bG12$Hn&_lMV@gdFw9LiH}~{|`L0x`CquudrD(QpMwWXPiX&TmYv-S5eFPI5O0~_KC)G4cyX>6Kp$S# z$JZ%#C9JP+z}t(dQ7D0UpB5f{u`^MPavBhWD-ff8~B5LZFj z7HS2BT4GGO}D~-M;I(>vcX%XcIu67VZgk+DpkDY!R_zK5hE4BtgL!i zlcCt@zw#@7s&76ka5kkJ)9a9?Pgyj=wyl)R#pKKVA_hhc_Q{4O#b%DP;-5sA5aPF# z)o~R91c?@yBG}xq5i8cj59xi_bwz8E(0pge8k~JCD}Sb~g*_4GDFLSocdL0MvHfK_ zGmpG?Y>My}EKfK8ZEa>(VsOj#ijL)S%V3O^-jnn~qY4pO0Gf8M;cG{{{xIb|1VjEB zO!;1VbleDW!p$}k$bu!O1BNs16nwI`HZHe;EGQdl))plIjVK_3|pL9aE^d$0=DxD=Wl4<(m7QRz;6{jg(8kW(XNw?KIWl>OS0`vH~n^ zZP6)yM54>{s>N5?oaeh7YuLu`9J%#Qs2^^>>?jh=hu43;P`&W-TGv-0nx|vHSwdr7 zL-kom@S$}H{C*QyFhN$mCb@9c7Jko@4r-ZkTtWFC3i3ndH;cTcc3~}=f29ISFZ}D3 z^QP-<76vym-iZ^ZdZf?}q5@qO-7M}SPm{wImO`dqK68ZsL7O8*ru27O@YFe=@`@D_WsQ>6RwVIvL*9oCFx8*-s&|>{%o$FEY+t#>R-Z0 zMqR>kvz%#DTN>wy8`_`+U|)ACLnkllV$E*e>87zYqORY=DM4-@S-ofeBQ6#UfDagp&G!TtRPq-j^a6@nqy!~vV zI0@nWgmr6od1okV=T*{DR+g{CVf6oD?=9n^>bm!F7(zk>MLeoW1f-=K zr5lDY=kZ`-ww~>!yFV+uHb-Jp^s&E#B5z=Gc`act= zuUmC~S*h)Plv{6U6&ZVUi)bH%wglcxy>K>=T)(pX=Fmze?*azV<484D1$}lpdC^P2 z&?Kid96GH`9IkUo6$imVGAVfImDFtGJ#$>Xj1;pIM@^p|`s1np1kM;(I;@T3f4x z68FrJmRLACzU_QdCEfRP=@ShDJM>PM)#T^3*|4^ZBlP*GirBIf=)&14$&uc?bg|t@ zfLx{&QRV(|@Y9uwXj}Qe-EnX`iZf95+H#S%s;x|Einjjec4&XNxw?FT6R=53L>5DD zamwu*_e+`AE}iS1pK=gJ5|C@Wul=EU&>F{H63;OLuYU@ZfiF<2FNiY>;QD^A1d7hU zt!jL-z5F`e_Rs`^tl^n{Ub9`3eZJ3CUQud3IE24ZQi*CmpWhlzr)s+=q+mHkRd6nB zeTXAj;zqOHTt3Ts?9eRD8>N_p7x;~z>Z|xrU$L^v#HXiL7U9@-thsbO~Pe0Krk(0PV;xt`$YLgd+@XDZf*@ldr0KvcIP9cQ5bLvVH2h^%_QG zTl)(d38dX1PfQlR-tMylMm|}#J3H4w@H7qNVP=4mVpWR~)p5V{6lWewf0}qCLVCKq zv}&<#QOzPE4KILu`?W12`m3c`uy_04(3H65~de`wEqu!bvbX*k{47W6-9DeZHzR6z7>IDwo;5|2tFd=>E z8CEsrl}Q{WOQ~L-90)yD0kF3}LfDNv0nJ{1e!Lgn-WCsw*52epf-gu*4D5Xmi-{! z;e7hCIt@h^Dr-=|n9o2w2S;O1ZYS-_8r|f!jfh{T^U0oqZBhx19SZafl zV8SYMAu;|T5TCW&N?+(ml(OLGh-*{H{C02U?h1!m$9$e{G_b(dojSz#mWLt@5?n{K zZ)^m)W|AmP&g<6_y8)#913Koy%HEeGx#`BXp&7-^**6vzHWDg+I3+yhJ(s(0uU_F= zTf=;0W0ikq5Qk;Y_Y%+0g+0$PCRc;VnMVd~jD(Iq3B2GC!Geg3VigY^S1-8opk2_|UibVR^*#!djDq?lpq!pz^1!z&vKRe* zaf+!Ov*qIj(nG5u41YJYA4JHS&&GYztb1yG$((56MzQq8J#-1?tB4n?7GztS4*i&{tS!?U)4tk|M^DJW=U{1ls<0B*_^jB4djT@%Z~D*KNzMhCBwS#bjOOe~8)s!SVsW z*1>qvSaDIl{~$*HLHUyci4narPie1}BKpS_zkZ#92}ruq(@6ibRsUps0fF{o{QpVj zx?2FG=JC9o9M)eW_AgIIAqN5%VK@KD9{$e>)wsaE%3-J4#}i!}$>2i8Ou#Rh}nTgWhFD zzK}MrGYXs4Wedk3WNZJ_vL2e~de)n}(YFMyT>MP)h^eHUR`u-N1kbY2%X(LJElvIZ z9Qc1k=O8|4O}SqezhL@Bq6SKbkW~?pn;jTesMa-4oEDavmqj|Ce;JmeWRf?p-y_ae z%!?Vwk`*|!SuxC3=M(_y9uXS#SLOLtt^`B(=Meta>?D%zlKcr5S^*ZC8asiQ(y3xxc%6_*Dqww5-dJHups%lQwu+EG67a)Y=U+vNZfH=7 z4(1riDJvI_zxiiKDj(>_4^Cx^xrHRsmv?LryD|e^f4AsNB7+8+cvE9MNb_ux;~|RJXfRQ zILkU4C{WcL%2g$+^3MQpZ>&+9CVq6Z9H*da=uplJap2e+xe%{zS+r?dV;_86+(YARg!DjX?w2P2UD45fB z>!bTn=8;s>{<1cKsx&u2MpUx~>Avu+laJ9VlD6CPy3Ff;_{^z2fspFp5UV?0o@c2R zsIfg4nFs`G=%28k0Ecv>1G0>p#P<1WtlJz8+u3jWc1AQpD)dSKOoATcl-HDQEJ)8u zscIRyfNb;X)hx(f$Xs!fI4Xkj?x-Ud*$%aZ30DWUk12OTu!DCAOUBG&?|O`c8CCBR z@`Rbkg5?+)!45?>rii=bKNRw$TZ9UCWuMTQ$@P*)DJm51o<&8lW-s26prKZFbdwN| zRj@|Qz`>!d<;_T!q^?5j_d$!8B6+0A(Yx?xUw)C%%yX)O@vHKUSE;$*{uZe`_dE6ckA3>4e^CJS$X4 z;5pkB=uHGvW8&uK`FC;GZ<;c#7u}dG+R`ECk)D5AIcIB*w zOk^>EJen2z+zWL3U$p_BZ@+g?s+x%l`=9ml;2_U|#97HS)`d$J?O!(aUr&OF0Z2uK zYBlbX8UKf=y8i1ODgXrpkHw*1|KqHgE0odocu@(LQx-jc5 z00Y85e-r(mQRiS_i^b0Fs{FThUH@B>9I(49qH$9GFAD*5q=NWt+cUghV`{IO)Zu=s z*kYqrcW=UGdvZEFC+_J>x>aeq6fS8fVOAJ_Q^BSkbO2@K zfd6fd6@or2@3SaRKUD~NstoQsy)Uto4=+3}1Lj>RDS92kLxaxN759`aR}$Jik(&u+ zoNyjplQ$F7Vjj$YHg`EFq-gN9jeW6vg!fNvmVe4wB#g#N);3NQNOM}@+BFPJj#OGq zq_m<{{~KtxS0GiUky!K3)y)xs>nEmBc7LJ})W=g%e6p;tBl8sN7#O%uY@$96W<)s< zbr)N_)o67&-KQU(@@X-DphtdriRDP~kycd=`dgfJRNVC!bj8=VC*)q1!Ao3rCiRaz<6@IuryBZQf$EK~w9G+#LZw{$2g}~{z8ORpN4g7ErFGmvMrP8p@r2d4W+74u<>N zc3x;5fU_MQ0xOb&KicYYIHw;#+EIKWzUm;P+vbdqX=ShZC`DGPMlKtSb6XU;r$RL= zXlO>a7~030^U+y}->NSpj5$|Ejvl!=)gTmwqdOAIeVQ3wERRr{NT+{@8=lnyRN5)^nanv@(rwzleD zgM1Gv%3B?2R8cnbh-ia(A`SY*D|Eu4JWplKHKssBwRz0tqA{+x_J@$Iz6={q`6fU+ zi{zt{Dui(~oR{pQkd_T}C)MEVMVdNxxQkt5L4X=tOMgk(Qp`X2YgN z?;AlI)b)iv6o~4C}){S3%9legy0q z;rXn3zt*&uJF*}Pxz6U3=Mf!hY8nkio$=T;#kp@-OoN7{SWL)0)2lk6{)J|h%ag(t z4`9Yo1J`~r*vV6$H`7R-5>Ca?Bc-a*FR_jZ(hERF;f!BIJtXDV3bTOaJ_C27qSu-t9neZi4AJzcAusrVVLz6wZh?@|SoNJ;**F|f zg1b*40x%FbT6_KGU6Zdl;rcw>WZZ>**JB1hNwpA?RB*>Z3JYd5GVxWOsmG%h-HxR- zj}WP;!CIv$9;+ADIA{Pg=bsle5G@U!|4DFH5>CA@{xh=JngWxuVqsXmedzj~kOzoZ z@A;Fo3?OMQ@(4z(V35=KZ2xD&)9a4JW(+usH0#!vRMC~)Mh%n-=n^tvev1k6EFQ!j zaD4qEX z_ZW7+%+p?m7G}q487FN5F^c24C?N|xV`&M2w&(R+2^lZVm>zvJ%V<-pWnBsxEH2O0 ztPvfeZ)NI;aIH~sb+CMu-D$9(8>ymLqrO$%I21XSdKgirB~cjDTv_b8#|H|d}zgi2BOe;l{TVa=>A8-1HVzy?D%C=ab$6uXr~Fe zQ`2+qtq_PYV8=*bBJ!NV^b7w7eih8CajFE7l*-(u?Nm=7Y#cHr2!I1H>Lo6{#H;`I z*+x)OpbS)|K4VPm>g@30)5^-sY)8kq!oWa>UZBz>Bb{bMgW-{oDFn6?Pa6?;o&Bx_ z01toAm5vPF%P5{5K~uR2r4KiM;;GKzkici?!-uDdQuX2-R-rt)?cPe;jQ}#dpHs1 z01;d~yPkPFZ(@$$%4hc5G7v9BA()$Ml3yiS96Q_U6JBX~Ld4cOwrA3XvskK$$F9>b z3e=Vt2a-!Lw}H}|#rkW;`M>J#&!HI(C&Vy|q_zc=JaO1m*m>zTx^8U~k8HWW(*Nxz zBH_CSg`v_0yBTT-YJN;J{pk1KX^$plx*Dw3GjaPX&hJizP>dH31iUmpV5GV5^_rBm z7QGR~iQY`;V7BV~Xc4ou1^nsq(gDZoa!HxnknOh-MctXZ_siEMuu30(U-bn$Bi*@k z&t$I+I|n=m zx6GGWd_qRg9cO#RqHK(i?&$1-porZl_oV?~Jm05+U(HB)q`cXk>-o)ZfOF7B@Yc(I za+Lb5d3Ov2+f;I*#E5mrl&(HPta%K0orWtzYmd0O1=efb!km9Dz3V%eJ$pJEHzQYO z9J5-lNjaLTFL^W^&(_+_WFS(uJUIK_T-d2MnCrsEH}N$f^RIPfV0bC+QpkI9tq-NT zZq#`t8Q!mpqq1);R(WYM-B0v$U?VOg1c=|`H_E*r{Rh;1^wHhrNNFdtQO2&XdrT)} zDv`Xhqmv8div8Y6ivn0Z2%NO^NlA zb&~1Gr(TQZu#4hkq!RE>Ka3EP_x-EwCrrn{0mz!C6ScH_y-(_$O>yPn2_KcobK7TW zQK@e>hb4Iaqdi_!T8RRqUE(}}eRIdA?L_wnyN^F}1IIAc&;Y%)STW^d=8;JDR&%-Z zX00qe{u}RJF=^?@ni{(8$&(F{ZsOWmcY5ilxnZbs&17}j?1iwMR1Mwb6X`m#JM;cq8$&Lq7XK+1k$v{6(WT*z{Mc#YvEk7?)Tnsa>N z;X`5_*fBX{2B-3d$0wrQ{kob%_i8XI$xx7dls5rH@Q4oo@!(D`k_FcwJL_M_YFZlp zz>1RYRT_Mc4E3?!<4gbi?U)z9wE~z5peP2lzcc;N>zmC?GqPneaPEE8SGMv1NXWm> z?zH(tz-F9?P2|-pNLu=fNE(KEsTdX$dbgiNrFV6N^fLy&i7=FL;ovgoYH}$mQHQbI z)@x$gZ;xso8jw!N3!qt-tCc*UKsIXJ#nC_RE#qB%o52>fB{t)TdrGz45sEIj!v*tc zYUpFixim(7#^11!SAp-K2?sw)qHizmL+;l1cW3*23fy! z68E@~Fb(Dx3^PnVGQpc|J9Z$9Pwa9tQ{^g}a)Zvl6JA&^55Vk2-8}XCmX}+5=T()F zRf4I0R7f#B!pa|1%SDY<0GB1q-U{^E^T$C!4O#Qq;tL4r-5*ZVLYYpNKXyOj<<;al zTvTU2YLeFlpYv@us{;9c8kBAOYe#&ZDULVuMlvnu64>N+Rk5nQ3|48^f}fP#DT=13 z&NK_o%Hx6N(b!Z{nB`@OR^#_F14fc5FIO|zex4_u#J0x~9@=r&>80jlL`x$$c(D8s%*j~!aQ*%qFUO5U) z>c~|Q>q+Z+6f)nd+gGCx7DM%NEZ4nl3hj0cc!4hC#03PxgJM9asD2rl!y)75b>iiK zz^y;Co(7Z4Ij~pb)9ZZa{i7C3m3pSRPIv?H$%wA)i5=<8`S7Fxz=*lOLtGs}i_yd2 zUo_yfKuF%Y)ftW5ycN0jUEbnznRg6cb8sOJTETeBR2tI){SjRGub4XY>$Gg?-qosT2!d2BM2f2#@u{le)F5pO7OEw1eAcExBmf)i4$UdE_C>jXSVDo3n=$`&?n zHBulw4WHo**S=%~+CF|ZJ~**h|4OoL*J4EX{ogW8cWJ~4c$x8IdEbogK+{e61!$w} zX|4?i%ss-gfTW@NHCHFy2WmS30LB*NC~sn+1#Nsc690fbFs)JZv=^*fY+WEO!^~b! zGn=!GxQhf?jWc}{IAwSa?9!%4>IiA;LusyVm1JWyDd&N`il2_poi*y-D+ZtbSp!&` zneKU-Moaui;0{NsLEFq&vF?bUN0LbjDE8t&ETbYHuTqed3d!CuCM=TNJi2cs&*C6` zTh&>`mZ(AofhK&kmY%QolS>zSVIYzivS?WrqNu0l>yse<;pNx~KwYbF=H~XAJmQ)v zYT6CrgA5ix-P+c>h)jXG04;yBe54Lp5D3J@ED7;lrG5zcq<)X}Q%X1jM$V%2LyN^q z-0FNJE0pInTB;w$vQH0I%|&xs%2!gk1BdrZFu5n7Dn|IFbB|&9Jd)S*dBBIDa{45v z=lvk&pG-yMK+noJ0JIQIcLEdHG&_7Pd*48o&MA1(!H^&cFc%tbE{}O!I@#(NOdcVX z0{M^%x8erI51TmOr!4e^wSr9_QJDI=Dho&L}QqgS1Mmiqf_82+*(nA=z%1GP_8I7Y~FtdVt@!J_kyIEnkp9^FRubSy~tB^IGbuNwk^NIAsoF{Y9+n`-jR zloAyIE-980{E3CCwDx(z@;M$kYzA$lrO>jJID;osTzT>^B^b@~rr#5o;Y>tmZ>mV7ihNQ0r}bgP%b)Mm)ye zm9j2ocbFCSnd_)pR$08Yj(2G=s_@lcb~UUr9wOKY3Y#`$4r!~zZ2Z((;EPPZC%m>x zi5rfYFEBd{;l_IZ&i%C7D3kTs;;oQnhqYt-*+v(M0|Bg)imB5SdD^q{A3TXp8Xblm zMnt=fz6rbu3pNXo>jNz(*x<5Jpm)pmyx`#b`gE4-{M@p_(OVfQL@X2~fEN8EXZj8d zLSlx~w$QfD;S;j!BP5ZuC9G}h)%8r^D8PKgGI}FoUI64e`Yufc*&`JdE^vox)h;l0 zaWQEx9iEW3XGozp02Rm<;&9vNC%p~Y-$+R4ICC1V?>?@0c}~BAE>Lu=y6TFhU0qbM z(7U6g|Kw?uC#Uj#8U#AZKPOtk!SkYj+BhkTYXl`#aSfc-yX4Q+(0yK=*N{HaUEOVuzsnB#BJvKIocTR{j}jTxwdM5WWl|8&M_^G?FiZ;_L4@> z-u-9Uc1_TgJ9i~l_r9B=C_HZfTBZX4t1~wRO~0O5*VoDC(mnMX1S7OZ=PwUDu}-j_ zAK2aC^Yp)wGyX^bZ5|xNjGuqg;SUQgLIHo^Qa|4@c~ZS`VpCa7oSX;fbJX9oub{$Xc#K7m#=NS1u&wRO?U9;rScTM;HW{~KCqq&bs!f)3{?<&X zpJ0`?Ks$99jDT7*gHPxd-~s5r#gK38{UjhAbQ@kzV-Og{BVF~xQn%22PrY%Dv#vw_ zqsFG^Jkv80uS8+*On{8M<6UC)_GTM(ABP&=TqW|Wu-#v1nu^Mna)a!D2y zZ@f$0)8@T0 zagcVro~ml#l(4?+wLL-J@dQokF{FHA-wVYVmK)G3TWs?;QJz0af`2Z#4|(;RlX9yW zQ8@JQ`={cJDPFU(*{}Jr>R&fIbIVicM;$wnZ+t>dsPxml3we*U3$(*NWSM~v5k2H7 zu~CyI)wLAL_o41)NXDKi`^?8L6B7-;&+&Zx3=3s!z6Ij^7HX81(GNk6CFyh`rR32h zp`6d!C3UvdUt#zxdpcHeNv)Ql?%#RdhNh%#)dU=#1+1r^_2s2c`TF`OX#a%sMf}_` z+?PjJ#r-%eR3Xa)>Po}iH9fif7mLLSSS+=6U9CldNTzsIJoG7*Mgwqh<;Yv|c8wbK z`iO!w2WZGEiY1o76JSYB?Kp*eaxUO>(T(6;VQdB^eB^5>`_HMcs8MoktR%yz4kFPjxwr z8!543r2!*#h+X6^c%Uy%QJqw@Ba1?z(MuY~FFn)t$#bWLwf{s>0g}xF30&sai_CXA zFW_(iPHX`t3#!+69~4fJr2ZyNTT6_@-1v#IYCs=`=$!NEZgr-5(fU=D-15HfS%DCHjv)|T8=rt=Kg)iHqx%Xlza|P zgVhaT`O!va)H^@LRR^%{N&<*%(W)2PYO3`Q7N7gOchj?%Sn|nn*E?@@4djv)T&9d&$>7QD(hYk0!Sxp zEj+cj0CnL?V&sAPsk*U@PqsSVbtoZEUn;r--5l5ZmpTvz%;Lfj9%3cHHD*ss9czV7 z8@`TJrXAdf4_InfjVS7wy?4>-HtHk>)uq5X)AR!zE;aEX8o|0>7_*@O7MY0XEeF|D zzzsUsy;k7OT$Yh{JtdZbh5CS~3c&RLc=14*>OoT8z0ztn72k-u2}EJ>*d}AtscL0W z%lj9Gg#l&JljTG%Xwq&gsS91n|My`P?vaR}QQeIl_-nH=7>bNuhdHq@sM~DA@V&h% z48RtUR32oxPcj>SpHW*x&uWw%Z%@@duuneFmV60P8SSArI-CqoZ{oOkN-<5 zOLs9Ua^ek8M}>`EkFqooy4w~JbE|?k(V1&l1Y&b=wyMvrINXSXtj+1w}E9+qeMu1#W z3U6;D;7Fn1V)M1cr@D*L^H$Uvi0yF%pfPGbY`Qxs^^?bN^4N`an>Y6!FhE(~8GKc| zSaR7^)8^b_!&Q?%Z*|6=R|9--n^7i^JDL}pE}n^?_odl?lJ;h$&_&>m^d`I=+(7x+ z0Ca*5Y%>XC)Apmq?qzEIO4x9wgHe!DwgK>JHP#=Z>kb-KEhmG}XE(nDgn6Sci0kWT zbVhB(`;i>vc1E+QQgE?I0r_tWCe~*~KKw$}&M>lx;QkzPeQ6#5R`X=55NXZ6+_@Y) ze$>TN?*N1X%-crX1Y%T25hj>|+yUGgu7_Kyuqq+*Dl_O9 zOZGfr8>GnW_+e@RI5jdRbkep3L*Fo>v~<~WZf;n#M89h;kT)WtUFL1Sj_jJ>u3U%w zE50wsU-v>39-E}Jzc*S?WC(`!+ zAQVRnjXYuT7v>HleNd`zL?_#n7cz#&4WaC&<}(tp9AZYZ`bV`RmS zXc2kt#EcYPLD#4!!ehIw5!~I9k%IjZcmnzE}gUL zX|P(roKeK`6}#I9E%&Fjr>Uj}z;Eq-6PnjBKdtG);WpJef9@knNxum`Hq) z^)V#`K2w{RxHuS3O?Jg^aWV1l_uZ93?;+9DczFDo%_tIXs$U(qDCP^ZVn}s>=MufI za@Q*Ynl|;8M(aqytV$sGvNx;FLGsu|YLzlR4k#$&LIGrT7X#X_wvKVyFYnYJ+WFVL z5V~2f25@!sOm6YFnd=p_vzw4lwVzi$xqa^0LpV=##HqKRX_eZUUSTWlWh5G`Mv0qK zH#VpKk0L<2ZWdvL1r|)0xQ}B`QL?!ZgET&4sT}{OOxW(IOZ8aAN+h1;b8T*NLp*rcw`401Khx*| z@hiQZdC8|+qb&$4QG5*@pyRhm<&*tIIU-wlJ?0ywAt{&)@whQ5U-GOarHXtHzk0~py6Ro3p9solm7U(0mNvxYj z;;D(K?kX6+(u3@UNgqIZf#`uHe`2afPR)5Eb1z9V4o3SIeoXYmFD06pxOSLpGurml zE7=712eqE?$x6GtXuHUgaBo2@{m6ORTVIJlxfijdQF{ix82b;y zcuWAFu@0r1i6qe(|M5k>B)A>*k)#&$+$>dL0TN8RVchh*(0euvOwf4Vnw`dlzsHG@=UJ=#{Q*Z zEeVEQ6wg>lZvZ^j4P8g8Jj^bUaLVa}=`6)nQ@Df;l{uI0XBVa9z{_Zhp0XzgQHo^Q zC)`Q1mpGCfix$PW`tx9%&>cj{{E*<^1zO&R1AgoU7`zW(3qSnaj{0qKpIfD8s_Vy} zBaEh~K8nY~em5`Bby}hO8PQMz;qJ!MJYo4mU+^y=EZ9JVNXJf`rQd4N4Nr{}9eN2d zrUd|pl&2=^ZZ<%nCPz)$hlU6Pkf6*ay?nl>J*19qd5)2NuW==Z-2}CPvU}Js?)OuW zh4+N3w<-<@YaT^@)j7Sn; z?vqU>DM*?rO<#BedEy4zLtQ*eUsYGl&!pp#sElnyzC@o9sCoJ&@^{PkYxtB*VSh6 zv8{>uhsA-)L-ec+JktP35sC*IgOTMAFjdxG=w9;SU`yDvf>E9mgpM=BWR-W@Fkr0* zFxKjm=l&c+Y=U$zHHW*$#P6uj9G}L5k1kPu)_tH4!+(5x=|a(9AK2!`XU0~=nmHDq zPf=?dQ@X-gEkUEzhWmv4(478PW(?M*mwWYI7SC>a<_&2z)~6p}d^{jku{)@6RJL1@ zwu7n;BYiB`sgT$E?H6HT!k7b@xhN`fttso-4DL#XY7G+A+x=ong)4dm={yY*8SdHh z;s_OE52F<++fKR3>@Is=8QyM1cI{4(B7@@FOU$T=_z=}K;=r!3=D13;^YM*eL! zOWNy^XB&Qi2*CY>&T`x6VZ%sy7#w5Sbh3IkB#eSh3lM*aynu0G&>4-4TRk)!ORq1F zpxT&ha!My}xp<~4_G5;_Y;O8d1&z&qb3D`bsixmaO8~;Rhq;UIs9PlP;fZ!+<=en2 z?Qm%#d~17U@~W%P5heCGGjO`J=+w-3epe#|TcR&=H_bN63$_rIiMB}h*YEuFuXreN z0gtrllH3eFJK(nfG2V#SnAJ~-#}B1*JraxPNm{Oyq$z;5lfa+ggmfCk@GBq9z2hL? zuow`ljB%!7atge>-Q0`7w%8KIh+lsB*_|FIQBdkxRYn-Ba-0Y*{8}>+byOvb#3ad4{ z@KJp9J4F)RF|+K4_k=|Zn5Oj%w#Mj7x1x|g*7SumiT%=gH;jXh*`RfZ>6n>Egd1xs zqU%6+LE3Q(`DD7@xokDtx_p$kJC)I*S1w*R2*m6!sFIZL5Jy=}l#KWI@gf#2h z3KN=Y=0JB05GAhhlV5GLxLnl$!w#6rZEi3rKYX6DVS0+LH`(Veh3p_cBty|VVcy=C zQ}@Dh0Q`m+iWizkRdplAfdeALkF+MGxBS0CDqt_5-^mblb_Q?@)f4(oC5+uLzpFb^ z7i{6qAO3xlIdnvHk1Bw*Qcjvd=A)@+p1tr6U_bs)WRE~VP>WK9@j`F)*Z1K-GiaYY zlWY&~dL+#(7Xjx#T(%C26IhrmD)m*@KZqQ4)~j@7e(Krk`T zkgHg!`oY>I9gHfEiMtJ05|!&Dj+Q z%0HUe_GY_k0N?{?Sg+p=0#V`-+zT%n_JhwUF$z_IU>efF0i2b<( z8D)fb%Y64rkcIFu2YmjN+z0^(1Vc+r63a+L(3pxszaO5PxW_~g(m!7eE9)jaXH&q* z#8^CiH~PfG&MUb<8=VYGWr^@QR+cklU5UhW@@MYPxFCf8xPeHiJM!}zQp;vioUkZy zd6sKWLb_N0W6WaB8=kAHA+3mpj4%;rW|X_4>monfS9*@Gms+~teGII_+dT%Lll{pK zF1#N1EJ;Lxyx#^3_}_1W@{I>#DY|+dTqO-JDhEsEAA0jj34R9!2P-da^|i{Mr{4o) z>a&k-}>-c1d|Kfc(FloF#yqhLm-iJyP-C)&9}Rj9&_^G(?Ca# z5?Ve~!>OY_(NWfs#7!>Sh89d33>x#!SA1_s7b9ZxOVs+OkpL?4KzfeqL(!aY$>qUj zWH=pg`$f+KZVI}XvM?ZbmoiVKJQz?PspU+tP1$EMXHAL3u#X30WzoW$(dB$Vlhsp) z|8^||cR8t@5mDYZw}hu&#Y#2qyg#{@nx|S3y1RR`^59H%dn=PfY%27mBgpEl_+=K3 zR_@ybr(j*Z!Em6IBg1I05(%22iELHR-i&wp@165<(Exf!nrUKzFnTNbeG_owRqv2t z9pI+2kFw7{_l=}y>_6bK7tc}5e)x{WHtlGl)X0i`t4V&=JCcGO(#H?0Gz7@W6vLQ% zz0LI&t$-gR$k5Tw)`>CP$t=m!Y2)O5(tqT-rO2<>z@$2jF z@7gVo73u&&q45oiOd2I*^*nca=&cXdY9njxPVjiBaRmQ4K>-tM3RYM-iebJd24>!? zyAgKp_o_z$9+Yprvs2OLmRUoaxx2)`z{4hphUQbVG3Vf|~-gRBBM!&wwGyiihLYe8kNCz1Znx~Fq=N1cL8hbDkB zp$O;wY42e`^Uz&s;15_QspQ`|FElQbpBVR}rTfUrjeCWVF$f+4RRbb?g%cQ#J?Me0 zNlw_^-EdjWMC{4}0@*ANAhKP$k>75JC1Rh5QW_i&DkZYx3_9amOYs6FPxEPiE_k!e z)nZOYv@{1|sL%t77rD8*NBn^NJ;e7=4IX~QVE}rekb7izw2XG&U;RZxOv-IScrKkE zhJhZL;qSZdc8@%(3iQZUP5r)OuQ<`;g2)9WD2i@jW({x*`+~VbucYn}dKU+`J$IBz zZ{b^f5$7{uemF=PE-7iuK;*m5D`}W0RUr*Ee#>h5zerfvRo{5q9Eg3Sr=gv2d3nF2 zIayvUsfF%c%(_j)xr7Y!*Lt+O7gCA~OWBYL!n;XGwK8-@CGa+3X@K1Ko%xob?1d7D z6sSs6)#(rCCwa9*5dFaXARsqNb=HeVI9%M-vRJ1jmM(JAV1sSn`L% z0EiZA7@Bj7=g_3UOdv33x}N$glL5fnrw+W{}AyN+5c3W8# zDU3A1CrjrMAk42+MO+)DOZav~-U z&5I9C2rCP}dpKnUI9fQ)e zCoZ*zcV&ka*+`7!m)i}@_tUp4-W;rl0kW+7Hg-zM9TVHo&rpvIqfc@(7VnG>6=%#p z{GXqXv)_FPmIu;1Nakw%~Q`{KaOq>FbW+Wh^Z@MxJkWJI7{gj<;Z6oRR9c z69_CaOu%7s z$WQiF*XgR!5HDEn-D5UN@iZ7{imp%qu!CH@)t_Ga2|EjeQuqZg+|NjU6j2mWQZPX> z5+Vo@OCe_W)?0eXv!oPv&_mJ}STaGAx9_H*+0wq(=DWRA`+>Cf(W6Ic{355_v7aKV zJJu#A=k`a&Po#J%;~jS9*PNyt?3^}SnJiGSNWn=%h=}|qYktOn6ODNBhaEH&Kif$w z&3?M6^>M-lrH?!gC+I9tDStAsJKV2}U?0rrWKg^^Ec<^Lq1kz5RP*X-jwIIzyEi65d<0JU1W{sm65&~*Q1JARl>(ZcBFihNzY3|wJ;^vV)@VyhFuVWdi0nluys`xZ$|upVtL z(eN}QW_Nt7nuUY%$c5{LqMtT997x-VF;X_m9O( z3A?(u=K~_55NB^&{$NH%wtHXFM%%b%)*yv3-e0e0EeEj#C3R13$pL2_yyAc|+vLBo zqFj3(Bl)|FWGI20PT^PSWN3cJKPk732`6ltN;1Fw^-?le$^&qQ$#KmEc8LydKOT+$ z`7cKvFhH5ydqnj!RF7ja(4V%h94^SAUrhkm_`F1<#(`U-bsZD^{ZO!$EYNy9!R{iG zSTKP|e%TK7o9-h_H&dI<=0Ep_eU}}n&%4S_Wk|ni#KV%{TPu4-J!o6 z&XDg5%~i)kmT|G}<*o0e+WI4+DSd}69RK~1M^{64@gW>f#Li;~Ga$oEXP~{>>Bi-< zgWcBb0qK)5T%UGBJjm|NNNb#Yc!TQKPS}4g0A4MNQpP^tFsAV_9o4g#A%EGG2QSI5 z?k_V3wEl&whk`{Nn7}>5plrfr{#4}pkbvF6)o25lKDeM{Ph?CV#tlECVMlwWYmdN{ z1Z!FM{=5&p1@eYp%)0>{y|pn<_xC*-oFR=XH~oTBg=b6f@I#kz{`5IV9k?1%=ms6{ zBdKTW1NTaQPTxCOk9xL~8>;g??L*}x@`~AAr;Rern}i%(36Aq0J}u3Y=sPsqI=`ed z#D*}VsGBUj_zo;s^U&?P{8zK)cc-hRnN&?!md-x7dgUHQ#Hw;(7KX~rW#tFx&OB$c zsYTy6!+%Uz(RWOhYu2B=X|Cx0P$y20gURje<~%PS+x&sjIg>kQmYd^!Lqv;37;nj3 zd}fPX5Wz=^tAT`lu(pPL{-M?vfP)qHz-tfx!3UovEzn=3V$yH_mscKDZqwCMG zmd1j=BBTa#9m{mavM~f4w#)WrN>4poZpGei-dl%O^O03dTAnDIbd~*>S1qszGEdz> zy_-S%6l=|SLBkjI`VyK1c@E>>GipvLop9`lr|&V_z@9$6_M}ws*UY|nb}ONu%K?dB ziBk_gj9XNaC33KpZ2%|OWSh*lmI;0B6^_Tk;H zs&HQHdG%)B?wjwhh70?*pX+VpEXTWRgyQ=~_L)X)2*!pB4e2+AWK3er#p++2G^ubF z+zCA2{CbbsUZy{K>~R^BefjMdqhFuU#pdMtadl`@b7FTFXScVvme6M4>S>QX%-3hy zF^$L)#J4Yh|1r07?G=a3*lrZ-R~@l-jt5U|R3M+dD-Yt%U)SvH9!jkw&0JeRp|Ayn zy_jXg_P)1z9e+i)0F&piJBr;(e|Ke=m%es7CW_fXJI;EHvFLgC5np#;akT{C9F_Dq zh#;JN}Ic{h*=Zg_c2vsPa56=>#*Hp7QjiM6;D>F`cpU z3@#x16g(c85m;WEeFrs+M&_Jewge~qBL&IToPmsC8Adi3ZcNsG_v`=mAyB{bz0>-U zyqfcll^AK^{*dZ z$&6K=K!&idMjU`Kav-CK6l*?n0;fq}(uF$`&fWR*Z@@0D-MN=%i7KQwIRtFSeJv`O7C^;_1v1#1*n<4`ni{(bWO)%U<_=zhw2d!J^i#&s5m>^`108@~z#Ej^rA ztE+uew=v|o?ni0imfYri=i1OM)_@Wi#~iv(Tl%({MTic*KSvk{6r76v)8AR@kbWt4 zPUM@)9B*$CX;k|U6sLgJ@6d_q+_?5udjJqvd8M3}3ns|Q`1+zc+(r@b6MroP$Ty13 zIe|bw*VE&c8iySY)&4RzY!RIe;V@?DcV|&=G!aV; za6}%Ay=pFa4{i0VvNlyblK#zCJ>Hy;v-F&9^or^>&+i}P6%3~1C|+8?CH0Y2g#v^~ z;*pv0>hHHab>`?I1C`eC2{$G=*^x1bcZ{Q>qs=k?a}3)^!H*UhVkx%V z#R=>wbo{+mftGr=1kb;i*yCTIYuc+BLf7!11o`#YKLIqy$&$6^S zC*PV||8R@nmhfy33W8|~x=CT-zhFDt|J;&pMR&?#&oYABeKl$omN%Ip^n0Oq@43$d zxYlO8N$0duFBx-C+BV!o4(m!U^k@L7uwRCdLv_f}!>(`sVpa|PzR6%dY;}S4Oxf*C zS7w$0apPS1KPp0w4{&z%4vu|%zQ?k+-{%~0?A3&rj8^+j_lU^?l&xGl0I0taPQgFw z8+fj0rtw~zBdiBP&)hxW$DEsz;l5?3Mtz_JeV-b9LrvdE__=L=^gDPUfcfp4!LvR! zu$Rm!SlfGNF8|?IK*7qTGF{4^&Pz3{{G%cAVgRjKdarAREqSc`#FKZHm!{U==X+J; z2mNM7;KEkGSih(9eZ!VrWkSwlNPswV&VbeK&tEmMlCQVkzwh#H{Oy51RN3Wo`%sZ- zUPwv9RxQ5mA3xsgBA{k6hDAT@+p<2t7vc9s*a^UKPs+~y^A=k*$iHU&a0rmG^egli zw=B-zCVVW%4J;R5T87P^wf;TLpNsUXfP%Sr$M^QXnP-dPx&_R5d-CERG5-5}yL{4* z0uS$pO*sB%_+MPMaa17Z$6%XelLdEJ}fNCQb+l6e$HcfS51PBy>b*dq-6A0 zEr5b@z%%=RxNPg;Qy_E?Y}UK7$JXdQydiIfrnllcIS+&tB-8RsAT>J8Xa%_5@}i9{ z`g?6zajIbZd{R?d4z@R~5x;3&_MogsyGneUAc*qbZE;0uv(BCr;5irfZ@BCC-9FLB z05*$KPdl@HiY+2Zz4rv;(f=nyjT?#xq=h^`tP8)%?)vGBFk~gRXuQpFL3gGE%b7D4 zzrn9jNz3JV8a`L4#B$pR7PoAI_CgCYnt!GO}Eag3{+XJ*It%zKwEKC*4{^FqRSqX=_@fXPE~|g_u=k$k#n7 zPefRxDx!~H9C0J%q*m!#XO-VoM^fg+>aQkUsaUZp^tZOw^uR#vEjlAoOHY>7P1}t6+OXUX zW^>6gAvy7dfeO6vRxw#QkgsnO$|WGY9HqXAd)vT47e*wfC6Z;*LU0>t z(+(=j*o6(tV*D9$-VAX4JS7?aLa2k%hI#%)?V=HcANHCRn-_t?ib4%xzX;&^2rJgO-sux7yLXk4{FJdea{V1cg&uxn5%{O_1fp zn(=`Q)%fL+2#Vg7zq|)Lu>$xaX#A*e6-;Hz3KPs@HP&5n`h&Px@Eb(^M0dwqTPY8tvQb^3#0OP-fE@DuH6 z*vaVPFNZ2kO-#f2PjD$?Dk2EM0)^e;`vp-pv5(sBJ*!;o(w$5Un?@YWU{@tO z%CBDR0KZ_+6TWk0Gw&pLb6}0oNO=_qM(MDxcR;H~w8TiDwB*gGhZlPk8^wSnTWY1c zw5&K;-a*gB6Cya~|H6d*3JQKI0PiZM^u0FdOZ0C>m3bmdTN&QCis@aPWA;P>+>d_Z7nK}{^ zF3s7d+Bc{HwzbfmgsrKOk#jnchDsw-7TyTe8J%`qa_<)numBn%zA7y*m1>!m50^sU zAMs{7DS%7wi4{VV)JSX3xy)fKoY>O@6h?-1!u!Gc!aGHCSVvPJ8?-_^+{kWyZSsL) zH4O`b&2hA;es<3Dyj?jtrHHt|#gf}q4qyJ5h$p`yRG%>#M1MArTyE@k!Z(}r7YX-p z(FE5mCUJqLvPvHq*36d9Faa24zjHxSqPPm8AKS1xn{7yz1nHW~02{oCFKmglh@c$x z&hDzn5G<7~t$e6V8mk^-i+w8~f#U%>ATjgClLB350<}zfAJkU0NY5}El)Cr_jYdW5 zQH9kFw>Xg=U+u9H;B>DN+cV%gz%s>&7_gZt>*}P$11K@w`c0ke)J+{jO21uST%uAZ z8``0YWx?O6MRH@f+nB7{=|oWu%y`Hp(51m;@Kuqan?PB>#T`O-B{}9bi%X|Obi-o| z{NQjQ=|{!`1Vi}qf-I_Vwg)pWdsJv}2`qbfXTe)|)s+_a`1cZSFJ~>y#@Q-I_3GbJ z$2SyE`V@;(WE~qQ9%{v=?{tafAKGvmBhkPzu3{Z(uMsV&OG>YzL_UIJt&OssU(6Fw z{vot!^N(}PCtDX;FeiZnY@e-578hQna@rOoITend3ziL4@s$>Ne%SC#r=bV=WL|$l zq`2+vSlBlhex?wo{~^F>L<4BFvsijIyLchxifpN)D2)tO4ILm&AliD)^U;pVx8B#( z5mU%6z9T*}<>6MbNsb75wVd+JbEJb(b6jbuVf8*f;W+MC9%voCSbA05%t&!jc^SE1-7Bw*uOP{g^Qb_R{@mO-hN?Cth9;p?^CF@PI%jl=l1ABSvLy+@ zhHlH%!tThK1%&@PrshngNpJf)m{*Bxli49#@&SCU?+YrN2$@t56aBFQE3G9^O+IcK z=6qb(q+T3gazFA}>QWw_((3GrL=Q%zz1oG6L(?#^3zT?`S?v$)1l7&WduWJ)to0p9 zyfZnE1lOH)y$QDUP$%cTv(yXlU&0nhbf`8;2#9fB>eQcen+rBqj9FHAa;*g)HM{Lymt~&x0qSbdrIfgY|h$Js(7Gtm!Z%R5?UrP?K0c}i$uvy zstHTbWqK~^88rqQfIM)b0+0t$qLykaiqkZ&1|>-zHxoZYLJ;rh zWuiM{5xUuy?+$zAoff8u$w(PziFE~uzEn{{8kPehu)*3(BD~95j`bFo!azK&pv4HK zKovSom2#I++1}6>dCzDXW?+HAqA51~xyGgWDRj3lWoJU)bn@Bix}sYaq+8!dhD%cw z**dy3!a5xh4*f&)B1=5(Tc8MCx!Jca2k@$%wXUloVuP;s&8ZmX%xHJ9PW^a^jg-k4L%aZJm| z@z#n>(}aCXPsDef(WXFco1E&Y+|}_MZMdSY!Btv)%Ph#dwE9O`)7ib!1d{EUw9iwq zJ&y17%7^BMqjYr5;i4iQ3N|xtDFyd?Mava+ms;14R99q&${<}!ZD9em2-m+_N{2Ed zFFo+|+z3T6Hw^X=Evz>n#ql)8WYp=_CF|(n>1PxW*T#&;-F~M^A(Np`BMTQ)#>CxS zTp&}3TN(LYdHUEdtgVW4$B&)sI>n^O<$M)Hs`It)Lck*k=Q~!J*;(&hG!lTfL}0-z z)hOdW9U{aEY7J1ox!k2dwm0?#d6A}J0;ZzFsxF?KYmxHocpx9$aXLf2K2W@O6V&aM|W0GaBo)N;wBMA0K+R zK4lFpi$x2flq)}tr?5ef(aw5X1T@EsMB53Yr-ISJyU814ShhRyBshO*Q?4_iyrJRT zaAK6mWPx;V!+TfeUU%1({!UTCia_DO-J4c(#Mz>p)^d+kZK;^&R|KYz5)`*A;24>E zdo;w#$Rlzx@m9&YSxi;|255-fSx@!ys^}#QA_jJMNCUB|&Y_`}c14!tANogp+>l!$ zsL}k~O%yY`=Xiy%bHjI=-j!E$BLro#h#34H(L$<%UENFcmVkB}rV532G%Q=0Y_uPR zsb=@xODA-gHE7-*$;ij7I}pd-ySt=pMiBwN?j*cO?vz&N5*RVY?eTFS4m|0n?+UoU z=6<4pxjOIeb9*x?1wA5(O_mmhRvFY)*l5xTh3jUg!FAIif@R3q`PcB0k9ujL4uA!C zM#JK)pM(Iyw{YExi7%2dcP4Dec{xu$S(v^#WXI+Cl?7kET2l=eZ=F>^fG`cSMUq$O z^uyyjuE@Zgh)b69VU-n}6OwF-o+E7YtpULcxswsC`PLC@Bw5;;g1+gadx`Wf0guM5 zOyYt5W>c#or`IKBLXXJGz{aO;86rH&Iw^R=9QbLTN4$1#-a6VzCK_JNbu;VjjRk4skYE`gg9Vqq&?F%yP~{bKxZA%|FYGUs+rIA z;))UCp>?v3r+B*!oqXIDOZY`^%uHrvu{H`wT=7`xxIsT?TuCqcRwi*~$n~O!-T0yq zsL$BQ{j~>qL=jwZvBG6d0^iVx47S9k)C5C`@)f0U)`^x}jsl!m1-#0BeeoKviaQvg zCSgXZM%rPJHsJBc*B0?=3CkgkEtsy1b){qwgY5Xn{_$QegVu!gWf`pGE$qS!;HwvV zl$0*LS88cqglS+j)`PKt-$L-%1oAR|-h->F-5waSDFwq7OZy$VUcc;y%UG7xef7(4 zQsxkr0iX4XPr`&WBFjF`N;c;Y0xQDT`pflgmgtj05Ahi+Eo1e(5eF7W73 zQT9SnyV7}c(>I;A60;)BVJ6Bwv9E#c!ZY3Y#AI?_8q7jp0m7-pgQ($DiRw`0gYvh# zc?+e-rr`p07ho8oSHZ`@A|$LG7kCKR7i*OTuQCj|p(s|BXY5@qJCjga|ui-4pARrQZ@jHGbeTOUK(` z*id4PYJ5tw)Xioh_cAG>E|6#vcJ zs0BnWJ1Oq;tR(5yTU;=d9p+k5j$bLQ5|p0d1Z>nRZ_*;v-c)n5zu0lXEBSE943HN& zaZWLtvDRzK+SyEROLqLRoZ>MHt1|Pl9^sl8J=+{x_LXF0wP0f5!6N;ODxlQD~r4IGGsMxhRS8FCY&ux)Q%$;^pveOTN*Af40AA1Nut#V)hnj-UVb zf~#uGW#F7F(L$l$bIRFy9eW!M-}@sK3dB1EIN@tXIUV7BH3{<9h9jYHG1=nHRD8nT zBY|B;-eNozU}5nASO3WQdm0oXYxF4A18d|6Qz4 zYoxRgr`PniwR%RJ?hh{wx5wFz&`_74ARuIUG6q< z)EZ+3m2c5MU*-1&NTBE*Tbq^@SGq_;SdI~n3Y9OJ^t*4+uUA*3v1H4d1a^BM$tZvS zh=!UNDI(7T`}R}o=-7-uBVuI*M5<(U~3-{Cyj?unv2HXANK zh^H%EkQSHNY)WzNdlsf<7Mku|>`bcUBD`Dz3-BR9XFcEzz4Mz%2iINFysU%Snb^D* zux=IAC9vRZgVLH#JzIo|P?=J{2SJ9K7;Q>rz60}1+5Mdp(Z_4(y_m0UK4FH>spV=2 z<}-WLhp}r}%oHDfO6p+Q@`eXAu6}YwO$cYqk3Jb(2@~>`8T6c5?siMXnBhn7Jr$h1hWE<{F?4bC*5UT%iQp1|YSxG=J-5A|U)S*_Nu2759!D6eE(a1xzHgtzO7 z=77kcv>`yucerYG;mc~jDNY;Y@wAB{+u+d==CTa2El(hO#3%hgK!ND_=mcFyW#OSp zw7ar0LRMgNWwn&0!J+LSq-xk-=vbE_Z2juSB(F{R$VyIUv~}%$aM2aj*;dn>HDe;J zv(BihOkciTXr^1R1?d2kg|MUU!=ZPT)i<)&TSwk(rkYLc-is`96=bhY1$a%Jf?a?}md9L+kmNzh5p60W10{{tulOq%Uq zU!!wOt%ELT8sM9s)xPKIauty|WFDxR8HPx=$K$f3&5S5DTEkiVm)s|BK{}d0E{zCr zvZBnrPSA3eDaI4R*}R!b%|#?+C zzV}5KdD?oqCt?5);&Euo&7UKYGZVSI+NkK3NRGFbpy`T*wz1VU1C#mUc(l;y<1{(u6<|gFPsxGO}~s;jZo! zdEfoj7&zQoTx{0Ntrvy?fxT_hwqS%dyBnO$eO|+X#w~~2Ku{v ztDr$i`*L@b2&5wmy!qWUXQ{2R-FB;48N#!C@RL@+<`oH)pOxUiGBQQPQlcsx;bkktAA7*G#>;LERoa%-_a_UqbljFB z#TFAwW{7;%4IdDxY0D0iMH4l8RepTg?1iN7#4?TvH(rmK^>rJoe!DmE=X7n1+&L{D z=s*WlYf7dvxg`2MZR3}IL^6s?RF)vm^TqHC&v-I3p-(3Mg_x5=fnH?vYI z@*T8XB|?Je$@kq>{xic$FE|Y&Qa2j_{8=JAA#YQ5YRkHKg7(}$cG7lM=Uv!)9UxeO z2ZFZ889=U3myWUNlER)xzN%wdKy4lh4qwm*eaW4haUHwxEr8*ijH2ia8ea$wc3LKI zv*6oJ zXrwboDI(kLF>JaUHr!gvMGd>V1#p1{(GB_S6=9mEj#-^ zdXvw&H17{i3t*6qWDb^Hc{DBSPSA!9(3_B+t`0M1AeoTpd0j)S+WzHeliv_tYL1JE z@zS7;dy`d}PD!K8;iAK_mZ}f`JPKJ?<|+mr8K5v?J*>Y)2(g>wX~Fhv;Rj#Vibi|aF+5~o`zd7gI!L9l%*~N4&l;YN z+`>CY$F=t?*IK8I*2GlTzUO#Q<_2_StU6Tre9Ux6^vLFTV(@SiO0}L7wP+3sH*hM$ z29;I^Nl%}Gq8Bv>@2OuB6-eWXb#H^6rMU^g#4}0sFI9pDwi>7OYY9q}-Hl=a%;M=? zxxV4S^>1;|(B5JRkEQV_l@}t3{7*#(Y>-qxeK0*S^T+9l$11 z@GvGof@RdwioM`b_WFhD8eM@4iP}BMZEO)nHp)>A=L>+{)di?nkOcOyTKUG3p@p)M zw|SbzYMv*diOzHlK(Idcg$0j9aGWr$4U9rp0!Jl(=k$9u5@P4gb0nrUU;)h6EG*rN z9WR5dhiHVjrI$@`rpsywVQ|pNuwF zY7d2HTWo#hs|nV6C*B|4^3`v0{JGFT0mu_E1c&pMJ<%WO`ZZC43IGY|dwTCb=du6$ zjwOfkX$8)4C;;%V^kVu+g>7$m|F73IuPym|`c2A!008+9(*RVuYn0GGZ%#IyL`txo zB+pf{wxY(4KS^Bf$lSVBcr5pJkLBy=|9X_87WJ;EN6PrFpXdX76a0*<(sCeo_&BRU zi71<*t>=fWD8vIt$FJ|15d3;#E>_&_woaD#Aiw`Do~;Y%^+dQ zikzzBGKoqo_U#U3Td?r}JPX1UDFLPCyaOVs#NSsFVFQt|8w+a~kvASl0sNx}~$gH417`FYN72d+dPKMyPGY0tTk)$_?BXK(U0 z?*63(e$yZ!@c?iEbOPl%nB4nXv9 zOm+c)$xdKA-w)O5oZ7DRR5^EZFtSMcE!pO3&4p4&kSI0ggS>}wMI>(mOfm>TeBm9L; zaY0!H{eqpOs%ZO$34cQbaYT>z2lsK@NazC+?2?HZW*$O4ywnI(8vn`Flp#sLz(cYoG5+)I8o)g1|y5*e)0$2g?@| z@H}i#PIT#FP^6f(x$jD5(U*q#a+KgpP8sH4(MV5ViEBSi-|pWe@2^VDmA>N*8$+^v zM^@EDrFw%q^&KCQrdhzR02H;xut4a~gI&EErwx=vy?XOP%uYeWzvsNA3faZK9^GL+ zVc?Z7mWy_SpP&@2B7dZpN(Hpoa@d5}>@8e+X93BCK(nK@URy1iQn7(0huv`xS3`qk4hrscHQdRz( z>&VC}^X0B*=^9eCMYR$eA$WzABMOmYrzsek-(pVU@;d1zt=#@LNGev7dzLsyYL4#17 z&_iGBMy~vXfc-oTxNc7|wB~#LThl~U0IU$mi7Xb~Z3btrS(_mnDoEyHusV1>9t5sA+p+G7RU9 z_FZi5vqN8iFJRdgmz{PKsIaRTn}wdD=?1!mJdQ#fcR{boh7EJ#?s@9ox;i3`1!#|6 zznWaCqv&mi=;m((3($St6r+1QNR9w2T!fU`x->0jmNdM)GV>jU&m_b}LsNxdW>|a* z#Zq!P3AIpL1EPF!w4Zgu)nnTtvI@Ed-66rTI`(S~^G+^Z6VJvBnDDu~CYEdhN@k_l zs%U#QfUS*ls{|W9XtUq2HOsni`s$ETHDZ(WqU16lPIcYZ7_tfAO6Qv50{OB_E}_pC z3;HtsRDr)v$^}zdwSjTGE!i5_R7+z>wWZ!#u&?E-1B^(>-sUg4vrs=iBkZ}!Aly3M_#(OM1if+>^6OEOF*rqgjJL^<+j_uF@}j*V#T zPrVS?7MDH-k#C|R*g(YYYdM=lHU&Z^ygXOl?odE>NyB6hrDK6F6l;)Rc_{UZ=$(nR z6W0wqzL(?>b|<3br~I7N_aZ-jnFGK(LuImCPb&fjzHwh?eT|Og`gmfLGk~@6`o@iG z6ueq4S*o|>o2S!x^xd5q;13 z_#Rh>>jtbN*-1Jt+pdTj)NSe6rnDWt4n;f{1_W;;6B2{I{uA91RA+w*9VNod==4?U} zRy}#75hP5k+gZ_t)=G*;#&oGWQMHLS*z#10%<6zJC;YpQ=f(${qdK!8``_~-+-=|_ zS}x__YUZh9)|nON(-o^n+;)kDp;qCg0#1t7#UqVBAMd9p>C+;32~?_gS-`ZmEDoJg zo0(*f2ejF-so>sAvTtbUuxc$oS72ftap85OIqz-LsxKj&&ld1($X9yM4f_JLHN4QQ z7k1CZewO7(k8}-MLXggB_uMG10iF(`_Ei^UM)EmcMQGO zG2@nlk)bFv2gFiNMt#}HTM-Mq1_U%Raxn1H@A#JX=Sv5o3}84Zy62mcLN28?qse*f zlxFpX1dBtY*@D`F9=w%dozMx{X(h$c4X>Y(K@Ry(9Elv?_|wi&Ri!B3C6AVAkZM;p zUH1BV8%waoqkg3KBC2GDQ|D_=a^rC&{$ulStl}~Od>Usz!XOyv#64%GWq7i%f6o7W zl7yE=Wji&*Mq`eHZ)BZ(BZD)~x4FC7E<*2HL?4`cxj1HR4f?KVdog?u(igyLR1M!&39z)t>cF5Y^#<;v^?dP@K4}6`a<*VRb{A zP=^Btl6h33BWX2R0DuBD5OU+t<9PUJ;c(Ab{6)?0XH!$hLMliN{%(>J^MXK3$*OY` zJ<`Vjn!^C0F5AViq?D&-;f6e_TyL5mJGHuLnfvs^*H`>1HJglnQz&7L*IDsHxzvyC zhY#goQt9=vho4(Tvu50<;Fnd)hu#xLwZ5hB8PhegLCeCC*{GEdSJgI4ngKr`r#AKk zo88>R{s~LdhR|C8sunR&@VP>WeJ+_4`PuiysBfRseP}d~gn;EGgGj#*pXJT$n*3>; z1s9L@Y}>meu=*%+dR)T1N%2u>1W-ZX{q0$~x-L{aLTuaHx?#kh0N=4+rjS^Rz$Y!{c zow&2@sSG%KW51i@)S&pljP3_h_o_x9Tp3EEurY=b_qv3yEb-L*@lO84p*jW0_`38fVvqy)h~IDj4}RG-$J`a^2kB&H~uCPP%ZotqHG(eQXp0e-O<2 zxG-sr{GKr}Irt^Fp&_GM)`NiH5&^35+|=qx8Oux>3r$IW#67n(u}E8n$}N@0UUupx zUGVl092CLXO+*l3RRv4h>sxmaUwdfU1|lZTc8bFL0Qj)!LTLSZ(fou$u%4wIyS6K{ zTavX852}IMW5Q^C06yQu7n(YCU(`?-ulVZcl=cBG!^ zDE`zMz8>MJ%yPqc!`X-n>b$j|sw0}SnJdo29X0~r3}SXT5-e-gLQwzm)n0IrmMb6UrHxkcTn}Ne_(-s#(~M2{`@wQz)z^2G3pU;)JON)F zvB^O|gN41nH>g6%wq+utaoWyd$JRFi*s01&S?hexYzbSH6cG64JIuye?!Mfz+8iw= z;;mAfYL-8p!>7pMroVC^f^K1^V6%-o`be4V!nkkFJVe|d%HgdGT5M8Z=SbwsE^;6( zbb*@@iM^f6cR7E!e;=McmR8_CBrcbKj1^o?TA*v96O63uC_G*`#h8t3>rXU_l|s zT*8?g_m$}gC+B%Ig$&%XAndlXCL1_<6={h=B? z_z5(bj5MuL6fptvM@*F{%Lmwn$qY&$kX654VMw(<+61NOobi+xWMhp7*F%=ZY}U$L zR^QQTmYRWC6Bnl@K^G*n|$9t2WfSxT$ z>cn9$!&*pP>+5G*8fQ;vQZ1O>i?##M3ttm=^9PU8-QZe`GB>#pSHuG^Xp$zhtW3UP2${bLJt+7t&` zA@CJL4}4gP%h=v3rsqe5O29JKqm>M@0hZWQaW~Jjs@VaLs@L+qF}CVym#2S*31_%J zwyTj%`y+F>all!azk=g_#F>ulArtBA_5==?lGf~R;IfMCjj2<3m}3sA?CwkncX~G) zqO{WN`%Y!X2wp5N%pF5B;mg1b3eI~iG!rB4 zAm^uVN8KO)P$G}#Q)IUXMg1>+HHQTly3C8sS90~U1BYocs79QA(s^@L$UjU|pPSQz z^bR=dhP$%3wD{g*brygH2d}kEjMV^|^OYi(_Z3Y;%Z@9QSiV#2)?4yuz?4#jkLS(D zWGT9;Ta=bh!EpEA}MYYIAPde8E_8KPuR8YHUBOt z2?oEEIMq$U8eoDji^T2T697Y6{zcP&jZQ}$Jx|+=xH1tzI0b&7ZxAktf+zw3%j>Jv zCDoZWK(O>BwVcbdPY}eQQs@15)?L}c z)}5K-ZQSqm3JkTG2|3yoQj3XE&;Ht$kOlsC3xHYh==`-Uw<34U$B$OQ$ZKYVl+|Sf ztz5(SV`|&>?jOZ4aZmOrHMX?@XR-*w>*|7 znYcXXKfH{68tG_xg?9q)Q zc%6MIVf7u8l0Pb^nBC^n>##o}GH658zi*AUDJkecG@xuT$9_~aSxKHqw6$({3+X$Q z=VeK>=!;&MW{>CaD$|+&G`=*UZdwz-VESUl#gCSA4Z`kTr?nUlK>xjiX9r4toF}Yh z_c8&(H+0@i> zZF-aQ(?EKBTBIu})wa5*7ZB@qKdwqtk&5-|CEM+{5Q}IQyTeeP4i zaJv6{BlK=+B?rG{1>2)A@&t?uTs$eVGUMshUNTi17GZ=KE1sNGV%sTN2macPS0Qf^ zD}WoBZzMDmPLhG6ko+15YB8gmloy9tnE|>1kzPpGNHI`l2oF?W2&RH{`k-i?2)k{F zrhNz6SuPD`A|Gr|4Cr3`O2y_eoo8NC_GvTOxE%)BR$t)2(VKgKBy&v+acgA|Ab+&4 z*Yo1Q)z3RpPrr51zb@_y00`g?2cGurx?`I_Ki=g-tzw`Vp!s>LMDOsI_8rjh2Xe>x zu?Ls`_RH2RWC9g)%#?Wl(sl{n@t+PLt*9#ixhk{cNw)v#LnJU`wFT4fqIKJ~>(8f^ z%LlSkK+BW6K)=QRFxo|`@fOyl`H!itf5`P~DgS4XYfpgV%U`ts{vB@sQt&^W)0~jn z^~;AiuRL16@_k6C7Qn5uK+ERc`yKk0=ld5i-z62YQ1a|5|Ld5;vk=2%mzQG)O?9mB^w&P z<&<%+g53d207xxW5OJ+CXW@26&H9*r&Le^3U##*|=zMqpvmGP%W^3wv_a2o)d5zRC zL*8xD4AeWI5m|D2)0y(zev}m#ECv?7tf}Lc59EXNfH;7}aK41$E6W!gjM9WWJ*Syo z0bu`)efW%fo2A|5^8_&F@?ZXx6r`#QQQohDuz)*pmnAy^>#J=RcHcC()q`3W38-d9 zlGx9{8bTvC1{~DYXayqJqDaD_YcC|UpxaaGkL6B6C--gBAwV7&sI(F39mUmbqp!N9uq2h>#%BOG z9Dj=%670gHu z*Q0EKx*VcjdA2HU3pRQ{%?=eVTRrvoxJp!Fumm6HNw4LO2mP0O(u3lnD8Np7AcX$D=NcjF$YPll~k>q@} z#}K}6e(W`<1Im1F-3pFpS3OWx8p<i=efy<9qz`;oK2waglS~>isgi1yZ{ql90FpA=$tm|J(WLPJI8N_jg|rFC2A5Q z0Ur?CNBLgBW`ZbR4W`!YGjEP&S{;O`f|i#U-I`~KSb_=QWWC}OLd%W;C|gjAed z>3#+2xa|G^*@!X$9>9J3QyNdU{d?OZ0D9*(Q1ew&O{-^{!0lAx8qWX^;C}4>upR!x z_CF~mZIX1rx!_7A!KA=gh?`0(GByPS{@zb!W!zK0O5Oe?*p1y0*C6;h^CidHK; z76uYKdrk*|NZsRouzJ^xxWeHkLj3W!Jmp4|yW0cqatR**g>&ze^fnnwh?gtv7cK!# zcG@@6gM)1g28F$+04#XD5r8v3hLa@sYdh4-{-`dAd~!?G-X9`ZO#^7> zzV2(=wq^7`oEOs|sWZ%tPZBmb@;sDd#FMl4_d!ugPE(x%cR=fG^Sx}yAEm;%ZnVli zBAq$7JLIVQ!TU$APgMIQygz&3<>}Km{D%2nCM2AFuYL5!vFjSq=l7ib&@CN~9_Z@Y zbI9f1o3rKDEY2LHM7@F)>)T1hSQqz`!x?(Y2YPyX2rY(o3;j*Ei64dA@|9q(fU{B4 z)BG?g=0DZme_qXT%XwfU4o&U&-t*woExj9FUUdC9qf%pI<6t;v#7cN(6aTB=#=z$A+x(Vo=IqUL zg1pF`h2QJW$$5w{&!j7jGjwkRvZ-{Kgn{BG2lGSSk9p>(qj1Q$DZ%IrirrD#- z7MYDw>!rXnb4!dZCK4b4W0HPgD&moN=>scYd@T=%EQji$QDtKXOFWq5jjgtG)%~k3_K4ZDWro86z3g%&S zDVq)JgAA&_?w%q$)aK;hVVnzhQ*8=unpQ5RRJ<5zQ*y7Tcopdv*x}CV71)K05XvtI z=;ej?h_KvTUtlt8=~vt<%gOqa5q>qsIZ*X{*GzMFQA;fMHV!&^*yrQYNdvTf+!DuE zgLI8FL1`D8X7$+~M@U`2n+UgnfQqe= z^tlQ2FFSQm%jfcrm;BpDxtl->vge@@O*^@Jr*xhMN)84ne7T=+_*ciJ?E>|ij;D{R z0X;y@jBq>dc=5k|dHOYp6#p~Q@2jU8fG=~?t)6`wJ$61KK!CIIz>R)+n^Uypr;+Ob>B@bV7I|82~FNcj&b{|iq4KaMNEoI0s;^X6;GV1d^$3b;|( z)$r!Ny0?xI4Y|qCx=QRQReSdLY44PgPp@{(sPNuS+TNkdMDKK_PwPB>HeAa2&g=cW z;-ZtUUcGX_45%mB^qkO{ACOQ(8}5x-`IOmRaW+Nm;e%nZ)$$=!)}5@9SyBF- z8pW6w@Q3cau-%~-1${p5Q&0>(#dqu0t;?1R9NGBjVnZ0e2)5jAYQ@%EQj)u;Jm_zI z1T*(pKy~XsZdd$rpY%?D_i1$pIvGD>(dWnT8=R*-G97W8`}RxXgN8R@QQY(P#W&7} zX}4u!u0%`sXTyBl0(UU&W9EeNTQWPA?tFk;9YWZB`L(t3rKO8vT@(XsBL;m=sEm{e z^p&}pd?@T;SG+-6Nr$ERA~A5vpnjoAT}?gNCUl7>OD*koyd(aG@gCRxqhsX~@j&I^ z@*0s#U?UIHNu_FvvnvrgW~~bHO50a(1pA3pwpxSL`|D%Zn@P9R&~@2Uyo0R@%c*y9lhF@{&H)U-$VQKsgWa4lW&u0*wiHXJEF>u!X=)WVy_ z$oa!fLa%`*)t$O0lx0imd6EjY>f9Kz{N-2SXw^LsU+k(;A5B&!=JQtPA$%#DA9M)5 zsri9my}2dDW;mkX-ev~7UM?mrjol~Z{862Pk&PFqMWMLWW%QgHJ5_Q?*?ODJcR?ur(S(?V%mhD(}xld_;45X?h{cg?oL(ELnPk)zDi2FDRT?d?!vo}udnEcCu zuY>#n%$D0+1)p)|moGnAI59f9>=1yzeZ1QabZ7D>9(v@tec-yI7G4}5&84|oQ3uO$Ee%MN)ykq^xMzG4sipH^|(X#GbcT@{ow z7X4nu{$Kq3=SDwWVD5WHyMuOYLUyXAU3<>30A7iM$&tVRvV)p81c2i*ACS9h|F{0j zYr3N+=_Q3}ck;l_r}lXd^xe{J*LsZKp&573=k#N_I;zhY8@cyyzw98c+&y4!fl0nk z^V?YF@9Qmh`a8fH*X}g!{zJ5Xf0gh7n45W-X=r<2+Z_{AKOS|Ux}m(IyLM3SZy*0* z%m07a@~@|)1CzVNtUJT`h(;HHwy7cXY@Lm{ED?3-T$<6>3T*}z6oQIwsi`+Mq%>3N zlIj^B5(Z_yooBAy!OG`Hfi1rOI!)4co{&39^QFyn>025q*||}Dq!B3_(VIV27stpG zqpqy19O^L}f7YxqrOuB~@rJm3sxFV#{Mr1H~m5(C>ggHor6q9WJ& zmX<@veTv()@6q$rNQHmL$or4mSXAx}-P5w#aacx0n4L_Eg#10Pg1|dcH-f0u4@g8* z`v)C$8ab58T@9#SzWcNITXDM1&9PkDh5*Dt`{38XypRVw+^ok!KEU2zhC#Pn*O?Xc z=tCxH)>_i@d%|Db-1Rw4%CL~EzQU-H@1YM@{~iW9fAjH*x%^d|?PNK6K+jlIvN#hY zg+JKiJ@i;P+B5YHrlBcht*HNOC=gBQgs?XcHSE+JL%?7u=Xj2v*{1k}g^n&j0<69j z743^!?Dk~Z7Z@>Bd)OWfcIsQn3VHqdbl63W5CadFTT_~;QFOU*ygP!0hgxI#oVa zN~dD_n9q`LNQF1roA-Xx%KP#t%99&b8Y`@4YFtAFA% zBH|!_4-XN3pVlam(OXtlZpD`U=Ox|$y4DbSNAub^XBbA7qP^chSfeI@Q)m$pEf6Q= z^2YZ?8#O$H=}lx?X>BjU<+}M}79S>i?{OuLndh^_YhK>Ws>OmOO3N?hV=s`zL3B{_ zLfUv2sZHC;cCbW7`CMMeJX9urvzW$Os5hp9J1yc@ z%*s3CM9)B%dgrPX)C%on)b|V+JFBo!dQP8HFu2f&^H7&EYF9DG=%3cn(OH-~r>=i1 za;K3nz5sZ^Au(*;Sxqq9W9yJA|$8`FMW68EBcpNBqeV>AD zLoLtFS^j2UP8$OTV>e^S?rr04W5}OFUoQbXgq!HAp$z}Vl5I1qd#J4}FP+&LYTC*D zkL8j8Lm9ZlcYXbDPql44pr83a5BVRL=O0S`!;?GQgP$M&UkHxe)q5--%@)NkJ5;HJ zMC)pb+ioiIsKlQJP^}jxP3(8N>N$?o(Id-tNAc=|njv0c9%wWcbt89&F0Z3;~8^PU^Mrg{G{%E6@XuC`OB3|#{( zD#HoQ9TrvaZjy|Gb$e#oo`fEr^XbUS^bWqJkq_z_b^%7rcXn)fP9NRf8{;#cBeYHA zj>he|0R)O}-wrX2in8n3;v)FnF`i92o;#W^v+yM};62|{-*ZzAaH+8flY4hS9Qt=( z-1>kx^6t=^!`Q3Fk~jTQ_aoe-8_K~+HlSdmKTn2Gml~jJ7dhL8j&mq8F8M&|! z*gy8Ue4pzutG>_B*#BbhJp-EBwszrdWm{2E5f!k}n^L8tNE7Kzx>5zC2!xVA5EM{Q zKza?*301lT0=g0Dgpv?x2?_xcAVEk7B$T_@<(#ei-1q$bzI*S#1Xku+YmPa`9P=5^ zc&0k%u?hiLz(KQ1>x~8)&)lMRp@K^dSB|}Ma&*!Pvhp~2mnT`$?Ar7r2bOPkIb-t= z<`4AwXBfE$uG|{*YNN81Q_t5i&}!zIJt5vnA+n5K7eIm%6KX zS@rJ2llu~YH>x58T9Q zLWVu%{OR3ZS%T zZF7Ml=VM2LI8YysDf;k>!Vtaf1!alDH4z6|UO7)iz+~4~Ud^EBFvr*=$FYoBnSJF8 zEo7gu_U+Pd#MZ1z#pviFQOFYkAp+H8c+)nrXveQ*O&Dlq@%LuTbXG$+VcX*yW5X$Y zOFFiO+vU+>M!b%>lNDW#MllKxb@VySl?hQE4TtUC{y)rlAuUzU%Kdm z%;u70v|UEUiO7uyWdCH8u9Aly#~+K0h}+{JTjfYD{ap|(8#M>?LeOP&0a4h9^9hCsU>?%~{-GRS>+q;( z?YKXXnmjiXG-}v2|Io?pDQhQ%XU=e+-~OKU`y}yx??=>Yx${;=yFutvkcV!B^kITN zq9T3}=c{#{ZuXR}6G8~0L>76nhhV=ED>G@(REzQTKX3kVc+}m*8iG;+*xbg+JcEQU|MY-A zM9JwBS8_WZJ<@$`^%E8lb@xM;OhDV9eQj}n_Vkq>{HI;mKiUj^@awhxR}&ufUo8jga0IlQ z!~=LMt1{!MEJhn)#ft=6!zi__3{uF-dP3~1B|bM$h=})p7xW#kH&gC!gR0qjE=O~m z`cRs$y95?@rH1S}0oW2lZCqkpyRwI&g(-2V_9pB&3`(pz0vlXjYrN9Ff`#@;p36D; zW!qmNfb4+Q##(ItnVgkoLgAsIe65%T5GG<2>FQw^~BW{P$@EZCKo`C7*_ntii{>dN@r6I_>2zCDBG%-9GKN$ zWJD?+>CP%6J!-0->pHWqBGDU8AarD#!ts3cyMBJA(3>Ayc4m7Oy7p*|st7WrR1R7w zE25$ucMj|=PhoWWfnwJRHLs6R!d!2Ju1ly#jX0mEG9`3D&#XS zTUv!6Fw1u-hhB}nP?ENXp)0Wi`Cr~W%#tVI7qC&s!w`kf#j?-IYXGDVkf4^`fSKwo{fI zs35Q=x!Rp6O_xK)rKa-JClu9=b1PjFYGJDPD=l0E5+lNmPM^*MfYEwIFf?j{WHPV? zrE}7IKB}@{f;S6%Z&v0R!>vp{Px`5P95gUhro9Ybfnhh~ZxqD1!ZW)nVn)h?!(KHJ zbVsY~Gd4G{l%chO?tRR=!O=vxXlY$0JRFr&(3KO$!3A@+4uxIrB z6C@K^t!^R0;j`*Bj>k8eh2vLYXFo2#DwI3l&2XV_lLB>eiHhje=KE{5mj=M*`jdIc zG#~e{L8$+jQdA*RA(1u==WC7h#@il7;InF9=fa_`Y5F4uS*>iqU`y^!TO<_^uNg>U z2K7gPHykup28kR&OYYipzE!3<{a*uTlIaFmv!XRJ0A^O43|O;sn}+4C_8Lqm@&B5t z0EaRHlje6Si0L_3O)AqL2Qt$5^1*c^@1=JaW^y=;t_cZg)0Q_al9gKT&-Po*-zCFr zf%t$e1{k+d8oPjPx=^U;I|Tdr0UWdNV!` z4kaZVAt+MNiE^Z}7#nm|X`(*%@jB>PUy3i+d(Vhkmta?8?7uqg*%OrQ;LhFE(Joq} z`dr}GaP|!F7vgUo$*rxWrtojui)VBwd7&eN4+7MI>03}V>Ad}DThY@)7rOu2iATuV zQV8Nft&EST9xx4c$2B&&yw^s5H(X|*+#l1=-=80^nw4&-k%ciRTFL{aSx#BDE^vSi z^x!m%Rh{`lE**km`F)zdr8j2*cYuy%$Y#^GHo7F#-O;|N9*SAzJFPc0TJ*D9e(hM- za}!75=6NxaZX*^$B?Vew1~g4Bxcc!}?G0Nl#<(IJ4lgwO)I1tZIE(QyW>KWgq;-t< z;NwaoStV}`vgtj(q;o;WtgVGCNfb0LzSg7|@y&`!&o(T)7e9LTrSfeeF|cKK_^yHLs;D$w9!?Vi&VF~&NdFbZ*^IR*32kBK@e)jn7 ztDrUa(9S3OJYzDCRS_a z`s28eV&at742{0@I-V_+en`P2#gkM_LFN20ZRQm{=UnJ%%0Sypb;rw>LRA(DtbtkM z66~_+(h8Y%!YWlWgVi9zMmPLN`xEf0{YpK5dO#`uL`u#d&+^lp0n9%h3~MNb3^D?h z2IS8F?gn$Hwai}ba+(A^!EQcz{>J&YOOA?+yLRw9w8w*KyhTa)kgG^hiq(S(J}J4; zd|6_?k+GLYL1v)3yvtx&($VJXl=bfE(e}y`6<66V(~n6Ro}7+rA_Xy}k+Z%!z&6Mv zC)Ai~f< z+4@w=91AN~JYD&|XqG7HO1Xv7!&9%by%N$oytmN5Z0~mWv1-OGPLS3*tWv+Vsow(< zV>P%VXx_33B97EWCi3Sn( z_0d)j3wLefEiqdHYO}stBss%&)4yCFrwzOB>2YXCpe5|S_dYAIIXy$XtGDF~ybIO{ zCD>9t-M%tO$9JP6$w`2wU#B%2xi97B2&w4cG9BjcO1fPbgSZxiqIo&BZf!?XDe{_U z#lTts-N`9JvTqGIk*Ew~5VBBYq=&`4NMgWI*q1KDydsX&W79Jko~^qSE0@$o60FAn zwKUXDqHL%rSl3q>YHXEfix!2=R+%avS$WVTv4@W~WTsmy6AZt}RGfjJGV2c1#o8?? z(Ho$MS4UQS3#RFD+JFeS%Q!#qXd^2PF>+EM>yxD7-~Y%2{#e`!%(g+X#nypMf&U<40eC)k-S_H^2 zyDijJ8eL-0(~EUq@97mX>%VZ$9lNH$aq~g(V-pkyX0j)xm6;6K>Elt)_nvqzxJ6?L z^B2AmPks%FnD@5q4dKA+&B@(XY%qy!Le4a*RhR_Mj@Nr8?Na;lBQB2^Ettxp+yK%T zaw;vdyP3Z;fsuyq#ik!YQ5;P5n(2PKp_}92w<2RokptYM)DJdOWehCfEZ!)2se+hU zxLv*2=aRu+sl^9WBi54kG3oG!r9A@@$tdAWK@l( zhMtG zB1Q<=9_6pHPzG5anE~d-??ESxHeh>RmRYvuP0ylhHF^_tezC2L#mJr4+zp1cX+{~R z9ZDdt6&M2wqfMR-6Ml2&1dQZHH)Ert*_-fqv;jU__1RaBKWw+qfH_@C8}5Ea3uogm zwizR+LnK;tFTvzwfE~Ne?}l*BCS=vWUR2Uk8^^2qx*y92WAMt?)6F;~u-WqVjW^%czn|KOD0cTX zw1=CPM}XHbZGlI1-F%Y4>fSv|MjrK`z^@@3Q{z*+QtWjlXn6>k_ z_SlR*Sl~R)F-zY13f5L;L_ygX*rhmWT4ya6n89T55gfkl`Zd|dYn4{MNY-5JQoc=) zbr`sAPyMuM9;k~SHnL??6t9AfhGC z11q(Ehz`mnBoxrl-X6jNTqR;zFE!@;4f_~y=2Oh>8mxWy)9{sZu)+HpswT=lm0>L` z-pWRSHVKb5FOmXr!c4I#Ph>ulIIds}(;d?o49I3L07BQy;AH8Q`Z384pW8)*C) z0tIJ1@^lT)6gU@RddGHBw2h1J zi!>0(SF=l8FnuTTl%%V=Z(q7mRXfl1LYHsLYOs8KMJW$0&pK(D8%FwA7#Ho z(Tqw1A?wM#)(gFfa_K^*>pT><-yp@S`0p*XtB}tgTWBR5S4*$eLKz9>&t*#VRoaC4 zyF_t({e+#%*@RVDY9K|tX_;byb0@WFHNS&R+Bm;;;RHO#>LqP4>4);zU&9-~N8{jHXxmyPHLBgJpF9?r?v1 zdIv%_+-8kE8ePI9l?u#ThArOBAKKV#cAeSG|9j)Cf72x(!8er=we8Lq!ESVg7l_{ zSPQi*s1+P8|FUvr;q`&Zwu+;RWSJ)SiY;2uz!|fijX4n2ibp2)0#6ftF)_E99Tb{5 zvSD;y>J1>_)zjtLvhzVH(0OBFWp=!BR+i7^Upa<$wGUY5PhLTv>)hM|$IejAF=iXI z?OGWZPw+_Iw|9%BqLI@!`A7khr~F9B&S8L?>ARJOG`~o`-TmTsU{2-xi5DcUJUE=- zFghgTu!3aO0xCy%M9K@9C7a9B^p(Db9iJWS$9Q0~)g8lWX>z?L@+cF8l<}TdIo?wr_Lu`Z z#G#6%KyfhoKDL6~XwBtE@V8LU>Tqw+O$5TW4aR~Pq1i{wE2YgNqv`36sHAKG+<_aF zOuH~b*tFeg7>zeY7VhF~7v+NVp|-1~RwL8iL`+TPsFU+`*H3*bzxjB=r!cD5nfpI}(~GW|~MaUpF9+OLLLk`e;5>;&@>LZ4E4u*MTiMf9ZRzO}Gp6w?BQRihX-z7}(UmIx24;Eww1 z^F3y^J7zJC4xP(x6)8QU$O)$LzgebtYM)Lzf6@>6eMIDr2AkFiDKF`aWx#0sLn%un z?_!GDSjH7UipEFAIO|fMX6q3F?mx0h?b??RkXKIJDQARQ-#Dj;!~^u>ZXZV+b1F-r z(ixz5&QFbdx=d5B-Bh6-0;S`CK0b8vF(>Krs{!rBJh za;}+J9kPi}3+zR@@1iyg7w1xU*2ZDdfhGmxYeUtWlZZ7h1as@8l$}+yo%Ufb&?b!*ChdgbXpwuOwa0f@n9+7F!>!q_O zOF>St_xEjrHq5IezSOxjk+Nisbmic0GX48EDi($30I6m~(02l#6Z6bgW&Cz9ix#0f ziSYQ${KPuypnq&%KRZY&bWHu2-W&YRx|A)H=Wb%05KGwF$ya}A0a#xJu4+7vI5~X( zKrpgsJB2k~+P(k!XNHwDobk2@LI&$YnM-`KOSLr)I5vsZ#lj2wCQVKR6KjqT7VGCy zxjxHG+?>cCJXYs?wndmz-p}OYQ2feVw%Z--t*ywY41g^QAEqQxsrw2$@Q3&$z35h) z1OD$a!6fr)#~$7^DQCn*CGRgYMW{+W${xcCD_M{=E-_Yl>q8lE6b&O|r>k+!_xeZi zN%{LM^yr{!cu!6XySy!0E%?TzT}pPsE+xByQ(B$oMXmr}=xBmSyK!@R-7~QD9sq5a zfnFMm6LAS$3$<_6Ws}~))i^!vvj!R?8y^QuFHpY}H_HLod)(7BFR#RKS zLB%dJD7_qvGUw&s(6}eJb%1R$VJG$Q{?_eT?Uk*PGld<{(Q1Oj$J|EVzQ8%FOGR1W z!h4W_>cFK1F49nAq_L6W{4|DL{e1Ssr&qko3`?=5g|-J6p$3yuqrMcCwWY~YoT^NZ zJf+2DMZRHcV<-7*s<74ObLPcfb?BddQ|0lY(_fvfn(CA1)ZT5Cxf%gD*{;u=#cBD` zPcHRc-&%@XbMgpx_$v{(cb8f_-%hVf0fd2%fW(`yloW!0kVjK2mo8|mjXqj9cmX8V z0-T86)h4+7w&-MDg`^KQc7@ObSLqU~NFi@5Hpaf#rleD^XCfktVL+x*Sl)lh%RL?l zyS2w)nEtJ?x(6wd)*D&L-?RK}jY0-c97^|UZ8?G9TrBe%4p)}^FMn75`+?)RmU|g zktts$+vIGq4*sp9?xp6X^^#_)Ow&>rFLP%?Ou)61G~{}Yw6U7pnJ(4Zvw6U$?>a&) zSZoXvm=X$WrGFe3517J;Ll-B-r>g{be^m^4#O)uiQHV+glX~_njBFK zq2@p#n&b1WZjV?iMXH65a_c6^M|_cFFni=Uf*lM1rUkb*4wrb8Nrn#EC^EZ}o>dyy z8h?4*;s8WXi?=6j$q5d7{*2y#69lMZ{uvmugxr$O$=x|&?P=`it~8dh1_Tcyc<3Wf zgzMG6%E^9>*kyDbxo{f8M@A``KDb)Jd6k8=uty`iIa{pDeVVs~3+jkoy}Y`1oT?iD zBxi>5pGc&wt{<%txKWjVQ7-kugYLjW=S;dspbcd1jdNb#5k5yN2DP-%|Dm5>J#g?N zEzo6oE+D-MfgL4n z4EH|8i}L~{EGaPGU&R)1T_Ia1nIu4ey z_GUM$#FhRe`_P^Ad(F^hCN86}3^A8R4lRLrk+_nQ1Wmn*tXiR2L8UMm#3r)g-ayb= z?}1tKGFxa8+lvrDHo;i0Dv(S)V=M+N$0vN6$Z3#OkQ36+<;&FRMwJ7gvr3Kkq&=*} z^h%sxL0iEEoAe%(iQZQso!Iso6IrU1i~m|9?{y2fp(AY*jqUSl6)}ElL|4KbdkJ7w z!g-ag?bO1iS<)NcUnIRc9cS-r!=P%4J4KO)UXjjd!>k{G7fF}${73taODi7+Okz%0 zwC6roC+h&y$$8J75T9VY)pw|H;hFnnOW#Ea0UlXYm7?vuO*)u=@eC=gxxxkk8!u1J zngGyz88c{0efcK&!kJVQ@nyv@uvpzwY`BR-f3Go)B@SbP2b*lMS6bPZNX-kI)S#X1 z;6jbs<1xA!E2)d4HI`IG8tH=A3%^R+${gV!AggLbm8Gf0oGnc8qq$0<1D(@xc}KGQ z8|gN|->s!SZCpx|b&jAS$||YzIBi8$#xb}>L3$(BSIoT>qM4$~qQ`QSTuyGihsj{_ zfLLMnF@g1m(<@9+f_}6$=*EV>ueSuPDd;VSxK%5iI5#u@^@BBqr$X|tI?78c`!`+W z88xw(w8(D~70T`+eAyK3bG$-Z=Ou_;B1b+jb%P8NMeSSpdEjLVp;q;XR~19z2c&{7k*=j%p}DlgnM zHau`Qwlu^vp;Do22wdTB@j0X8wehX-j+y1JsB z3#m`@k)+1AvZ!fvhn}`}0=tb>Nf>8r+G9Swbf?*HsR8+{VRDef=?XGO;Xb|!cs*&d zD4Wf)+96liM%-2%?^`~azpr0rQv7yA?8V10vw&(6kF4j29hwJt>|E-kiSW+5To``IJQ`q}a$G?LVH)jCm| z>H)>mML3s{38HsktLRF%n^>xIs;;5p1Qi5@Y%CPAkY!8eO6nm*!d$-7YYV6Q4<;eB zEJ9_PV^$=zerZ8$C)-NdyNoyr_Fx_tbax-N*RRCx^uT|KoAder6i^K}rE@jQ=?7Pw zx2U~&F9*S1W2~<%szfiZs6BtyLXR<;+`_AvNzN-Vt0cdHP`3OTcfa{$4MV={h9|Rf z^KVPRrB7}2IFuj0y4SKJYQ(N9+Xz2g&%T5#?r}Dmvv&0FqcI&?Llqqg&kr#keKr`j zH)6FdZQXoZ#A;k=sJKO4b!9t^L%7!jt+n&bA}v@1@j*1zRhzevelEF0(FF6u!P)2^ zUb#CoH6P+*Yacz_S<9p6AmL|H|3pqMo!bj=F^pNGEx&lqkb=W*ygujmEKqkoeh${3 zkOeg@AahayLm?t_`x2(`>~+Q;$Tk>DuSrI$!_dqj@-^lf#kB1fY#thHx1}z4B0-`# z58}8dFra7^x|IJ4Av1i44=scImFAUYQfiz8Y(vvNCnHVE4jxg&^IMkHbRu+ncU6e)Uu0tu#=ENuu@5BWRVaOw(ueE_`Ty7y^@aZ zp!g*c3%PD%d=N_`F9rGiR1~?_1f?8i&jwB>WzPA73%^^9v_0Wc3cBNQUP9M&=l#%r zb8^;A9oi}DBJg>XDH1>1C4T0nO+6K~3tIqW4v2}~eW{>1ldI?&wpflPm?E>=#LUJG zj3RPHMY7H*X#0MTFG8Q_<;v&}ywPR%e@}{bn8ZMYz_gc!m>2?}Wj`NcsFc+R5<}y2_R< zhN}4_WpGpBFnW~-6x%M=vU%sgMJV^iQvTxkCH<7cQHd3c&)(^=v z4Y#+qm*(7Wl~!zItZ@{>m;s2xR%o*}iEWc&g?o0<+`2vYbMIsxS|;V(?@6i5hxveG z7Qa+Y?mxa*k1=X9_d~vqlw~Mk_8G7k(N<|x{gjmAIXjYquUs5^O*D1=Vm7Hi zpeT(k#aV)@C9WUZyP31^h1`I=_orjLCoCGh9ak?Jb2fjeb;)xi0}tICI6L!_lvc_e zv<_|Y{Z3+VD*0-P$2fJmPm{&R6XMIBh3q0s0NmpkR>7*1<~RiCzpEoaOD;!^j0`an zCzz>A;|-oN<00!`<>C4{xre`Py4vi}u;<}6B09(FGT8F1b0YNmy1SG6vk=|=lk)e{ zzqVD#g!XfXSh#~=VZ!J{G;GAMMM&v6I(r0dBQHK@b z8sWFM#O6104{#at{7OKs@1|+4Ox$FcN=SMGX@X(|ME^iR>`Y@eD7S+PYIRK3+UD~l zH1DDbO&d>Va5VWPOfsL&F~cXfoEmg&d-epgls<3|o97yut8GQ(M@f(@HI+sT7r*o$BVvg1B=q(+&E{fT4QleefLSV_|Zt)iqaYNE!-*ML~M zn$*RGOKh9yHrbZtGG|JG40^dq!sJ~E;J|Zm%GNZMAYZ5c6E#RM6_nU95g7aaxce+@ z7bcrbsP-xHaN;)&u0Rxhoe0T(=GvgcDg_yHafZY0S9V#P;U8$5J8q1!YIg0sE^)e? z%jUkGHq7!GK#8&@H!^Px_jq6{IK?S1Go2G!sNJi{2c4X0)9c?K4A zoYS8)#JUoaI2*Rt@L!o=ugSJg7G5k#;&a*-X4+11884 zAsY|OZ|(%H3L7Eyg3sIm>9llLLr}VD(p%8RrI98RRQ;g$++;)NGrDpKxXN1WY6SiP z49NgAPI`u2&pl8i&b=UA_P|4SL{=Hkb|4xDk}mXEdbT>*EHK3VaYs2SR?=mdF82A) z;Bwccrw3U85+7pLVtyefJLx8|Wdhd?zK@$rX?CjUaHZWd+7ceo%xFHQ8Yo6$KH7kI zYR*NA-auJlP(7Fk@JciL1?fi)VEqM}1O^~&x=bryaQIX#WG&V=QM2hs*(X;_0=K?v z!Y|fmK}j6qZ+DUeA8luCGRE~>`H4lwv2#L%T~5Z=2dq6kB&A^xqL|ViMruByuPo7k z0GlqWvHK-N%Y$S>CjlOyfnp0^`ZYb}8kGoJiSCIIxp`(h<4PM)*zUOfo2)poLSPp9 zrNJ}%@MC99j4P&B2@%;;FAM+LMTULzqsQR{D4x$Iq@rQa+vE&M>)@+_C4u;nJ!V)* ztz+ezACSW{WO<5Asqb4VJ*0JF>TcU#(j0bXA66xDO@AdgTa9HPCeUp%U!rt;fbFn?5I5m|fu99R$7f1fSvH>)&Nt)~otm(dZf6f=Cl$1JN>8HW<*Skjab+n+w-{*gqZrPw9Q#mKB8 zWEviYb#C2+dym|&t2>+!X@c`xK(010E|#vo8=K9F)ODkoR~9Qzld6Ys#VE+^-XEe)>aQ+YPJwibAPDAUOIG+x0(S^n>YJ4bcyY0^)ns z#IA2AH(}zYmmF7TbqveF=?>10TG`u^K?gmfTWS;k;GKtdp7k2i=ed4<6dN~md1|*1 z%;=fp{=;HGUBt4};vZDpr%AuxsO7t3cmBQFv0^M=ijILQh^tEbH$Wlh#feY^hy;{R zhS)_+ypFqKaCCoE(uZB51}rBt7X~SbM8FgX9$~rdq{D{ycN2D@nx_VUD7aYt;k%4S zo@2y!$3i<#0mbp=*k?nhm4Uj2c1pT|-S1a+p79#O)e0&q{%?|rp36h_XiuBJ)98U0 zvoqbL9v|lGcip$}6J_~tS;Rl-!l92&0+n7OX7Q{4;1B;8Yz zBLDU4;LU*gPwnj*JsaL!e@g@If}WSp4!CPhCJR5xMViN*82_mo6%J(9%P7*nTZsaV ztS_GwutpsHXrL#R{L^(ohv(HxJXA^b4!78r`of-niTRP-J#~uz9%Q&Kmfn0PJPH#7 zsiZ=)3`I@469q(g?w&rULBl0se|oSR*WN-xNl%>_En6}D>Ho3xRR7Cbq1bhePy<`} z>f=8f@gFH;e$lUJ(_SX|U#RH}8TZQ&NNjv=W4nUKDTtZJ$ZsuP< z{qeQ`&1s)%_ytIZ6TkeJ4*w4$EGinPT>R!;z|Z%mYx&31{dJ3VfF6(jv9#phMJ9jy z6#DH7AVl!qN}&7YUqAidwFy0P?dGUNfmKPOfRV-B#7i-xPtlV9?bS{wgiZr8t;=7p z#pkKN|>rdcc=Om$L?4r5#eVcV8ZO-3lDV=;?v}*QYBc~5sW33!+J5W3^^gDj&EC7i^ez9c219{T$A6U}d-p4#J#j>W<-c}t?rBZ_?q@&0 zb?Cq$fP|-2VhZQo)o}x0`tTW*Bq7r`-ASeQXOV6wUE)YVQRXmN!F*-$$_9`(79Ngp zRqZ+J?m#WBT4{?db@*)08bP`~^H;#}r@>Bq1zz>)=3WDZAA0YDZD>ZvtCX{X{Ocjy zEkd!qnIuo?dgg0@L+=uF*QiIbb(>M+;KY?6D5zpt>g3+zn=_$b?keT88F5VaIN@RE zg&$|}rQd*#QeQui8}P$GQMq;`XtRr1?N8w{`jl`%Dmm(mP#1Ypp3r~~ky>c6t-DP- zyL6(fNM-aVOW5532FEL_N4ji(zmJ{eRC=j~R>Ef;NBH^vwecidj#pg?h1!N`*UZbvd*l~ai_Ly z_53INDl-E=kM4>Z^d!RR?}r-!`TdfjE-KvTnykPm)h?JoW4|UZul*Sp(1b2A(P&|M zgl9s?8!Nr>MKcK~rNqyCGQ0SV_9TJ^C_=i6yY>5y)^2bocs8QF#Yu>aeB7a94jnP> zD?kRQ2Uz&$m1Z4N){r#RzIAUql2-(h0po#O8JyjU+V&b0bb8$T2wx_!qiyr zU*YDTJ#^{tZcl$zBP9OVWuBp*;$IF;eM-3Z)__GFG*G3sYU;iG0CD*Fi1;{lZMq{? z`{gN~dVm}XSXp!Fb#QMTwJ`l#-D1zqG`QhD?4*uOIJ6!D5W=L zq+B-zjW#wqm}O9&4kFB`>&%39Nua+^_bDM1e4Iq%E}PL4R3nERKPL15Df>H8_dE-Y z&GulO`Yi6=z8wanI}1{%(xEn4DM~g=V<2iS1mUrMj_Ccwzeb0}iG|92C8gvSFT4<< zY53j0-Td#LP6Y3LGU$Id^W={n{At4npB|WW>G{kroxM$djp$ypvdyt9>Qv4%-ep-v zpn$6|Do-O}_7E>W129tB^iIpH3|l_uTK1cvTyr;BF!jae<|^w@9$gLCD_!NyZb@O4 zT#aG(m0o342BQ|4z^Svdvi%#Ja_Lw$a$Ydj;$pKKHLmt!dNopl-u?c$U<0|lq(}}#>IUFZBSQIu zu)4^Yi^(M>!!zGUqx0#GTlu#)qN*W4#`HK)XX1n}E4z1QM)8{srSUqs{zfk3jT8J~ z06;Z)xeJK*h6KT#PyHNa6s6b zu@zgxy|zJwz*VchVO_=H=50LIHZ?GRYe3OBmQ&s-O3Iz#KpZWq*$A2YWa+qAo;OqS z5u5tZFT_YuorQJjDy#C!Q|4)^gepkin8j4DEF|7MJ2CMat|GEGE1<(cHfTMLO2Vpr z)_Xs=m1(GWN9?cN(4XOj3ia?z&;c1RDpumh3O>8r58Asc?+r)@2$;h%OCwrMcBuMw zV9f~`9JKx1MF3aNXn5?cti8H1m;Lg7hNoE&55T$+YruII0X#F;VSDOxHd0Sdr6}8$ z6?$!{-zXFn;SJPXY1b5r^afr_7Iv9s1m?fia}gXa>W|M3*)T|#^q5sb_4Xc>CCu;| zmiW5BjjO63kJgXwHwVRj0d}5E|4^X=W-;~L>lU|ej=$Gus~kPNINqw~)R!i@%O)pb zhV#`ebcWfyx5GMC9|bLc^j-x6FiN!#Jw{=CQN0IrEI-3NL6c5J4}X7bM#@|S#f|uM zuYs4w$YZueSta183LB&$b>O{-W&N97mO#X@1;U17G@&gP&o*f(dHM1!UkbLYOqlJo zC?K$VijB>&`x(9XE66{6NhH0&5GAa$LtmC9y!RO@_g)^j-ACDqs=p6YlW>J3rUE(LhjlG{>F863S_W!l`gV+13NB>_x{#q0wvb_flsFy`N6;Yq@~@!2t36pQtfx=egd-kf0LM)Df~Ru(0obwr6nM)8J* zGs7q)!-IO=-={};mk3CfyA7g~JV0K1bGePaGNqa@5tt}otU#~(zTU0qO3zEy5fEZw<&4+K*uJqdJ^d{oiaZFMvMkokQIsvxR@rE?TdDkbyxv5# z`EAW7_>yw*hoU*-mLRpj$Db8s!RznJOmDTWEy-!FI*H5xEyjQPbSF*1=i^R|Klif1 znT58F?9+RnKj*qc9oT=1$ggwfPRXw?617jBcvE=k*RV71`2{u4oDf~Al)P0DWmeT} zbEZlLEp$oK^YRJTM|*ygm-sH+lrdT3toq^kX8&A`tWRNWKIDX5i?1q72-Kd6t8* ztSOu5HLyQo0aGq^qa@a|m3cSf=y_3l_197#i9s7HJr_!o1MuR*)@CL5)z0?WisE;c zUtg4sPbGey^Hgax{)J&|7~g6)9^~|DY(@VMD)V1n?O%iWkD=~xJTzJ$Wrjsc)q6B* z$LLnK8rL%hutQa>aMrVZ8}^pPVKw=+IQ5M|wLhLK5WL85xm6e|~w-Y|F;#nYeT*QmUYzBbb*cLL>Fma)_lN6bSX==zmSZhP~vlR|{hpXbZp;F2lI=|?mx zdx86Mp8BdctC`Qsf>(}=OKWNIISeJvJzkjqO}wDH-hakJnw_zXSs|o%RRBY*Gnjl1Y%H=(&A@NtS;S+ zlTvP-m&|HoFWQq*dd;L1cMTwa-@y7g%9xd2rjaDyA>Jqi=3aB|pAg@{6y!dOT4nFQ z(KGKcwyDgfvKXMpK3WHf1P)cU4~s%P4BZ>fbL0EFQXrz4-|IOj~_-IH_+G zA6s19EbjI>s4FM=8)j=^%9%#0n@JiqDlqLstPq`dPyl%N>9iC!{4l$8=-^l#sboPe z-fRBt^T_a*bQ+Qqk#7DD{BnA8<49+Z81l-J{3K73;KYN^_e*C*1aG!zkJY<~0W?1S zTi?TZ2g*C(`*2`hFTb0=zH*9e=#}^%P|gohY`;qLm#`$mq_?>qO+9_+O)`$(q}Rj_ z84va#XS>t1XZy0#3VRjv%!ki9nSU|&DNJAN?9{AdC#abvt+%dmzJAp`-My+CK>un@ zri^Jd6;vV}llzgr#Q0Z13Lw8COm<~;BPo#9AxB+#FLx<5H9+%xF;6J1`h6Cn{-omS znoV7Z3tkCcBS2f!cH=fcW-b#YipoT1=Bw;o3yXqPZq(mD1UGCfamjm@4)!Yra#I=< zT_;;KG?=B!{k4Zb%_U`>ZYhL$HPl!kNXFV{r}2k89(7og#%xtuZF9Szr@UKLH5Brb z58`lkl{p24or!uj08t3bKA|F`L&7HQmpF2*|V-DZ!%9d z-eUxLR|}yOzm~Gny+d4zmegNeba2vHBMs{D>ZS@RGJU;4E@n<9-j$W~v$x*LQPG~o%byW?iTY!`q1`j9t^Gvvg%z8KFe=br-izs=QOLQ8qyrnDdPJ6PG`F1H$MiP z(LW*svE`CY=dary!}?&!=%*Q_xp&tW+K@4owF!RUvks%;1c$Db@L5?qyC|)q80$Wp zqGOm|y%5I~WBp+{-{tpau`PhOgq4;Y$WVIcjhv;pXhoQ0bKM(@bQuF1OF6tTwg*d! zRf;L)$bIsJaL;0gA5hK6Gz!&intpAfeJ7-BIuy6Bz@m;cQgDaxQkogOy)nnlD|Y2d zWNEL<6&^zF;Q^a?+mP|MdlA~!eaB18ku(z8-#X#MQ7w_mHbdr*#}QSD9wX4ZbQiyQ**wUgyW3ugnF zlQI}w;sM98nNio5U#^PJ!cSI!ol#i{Vk050VJ~BkbBe68_r{;~uS+&^owfH~G4BX| z&%#$;W$xSrN#VEoO-dc<>y8ywytX|B-5?fGFyilD69&HSKXdNhjh-=vP;|6VrPFg< zei+;{V>DgoyiCHC@aPgnOTNyls_N+|N59fy^L(JdQL=4w zJJK^BsAq9HIA8B>d9C7c;An1g<+0iOTWc4Fa<+oS13q|}m*Co0r)^D{sb`b>civm} zXDJH#;|A}xY^-S$M`+s*5J_#WWl5gz4XFMPc0om|5t~DoA!_Qqe zPR{CQ*~%Y7Z-?`HD2`SX%BbxDmNxWuPL0CnFAelxr1He?oG(QR0}hAlg47+dE6ySv zKia)ty~&R((1_knWYl<%o-a*rZuNWJYuZDp*qTxvs#d#@Cgqcpo&>%HR1yrLiChHaom_2 zSlb-OdbN1nw+4EaO89C-hcjd0Z&&9Ql-`(%jHja-eY#eLdZu-dtmy4RcT4H@o{=0ccDehR4YxSuYm7!>jL+1Z;xNmN)Lh*zM_G7|W05 zmw#*8iHp-0h--a1?--0^4ROQ;&-;ZR`P}*SYGiYhe$PP-O<3oI_vWYXl8$fQ7@UHK zb+n8Xz!&*c<{{1cLf@&6(31QvQG@3EMdf$+UWATO z>LH00g3zTyEI|ii_OT{^c=X_^>re69KfbE)o_HVs;n5@BbIPhe&CCDrwW0(_&y;p+ z&;8Fm@$02Hj{ViX4x37?-@E>|9{ndJT+W&L|_0WEN6vblM32^}UOWqj(t(eljq|Kt9 zm$$MG=3UwQ>(tp_IyG)MDd}Hy_&WPbm&Tciptt)we-r*CF57Nm^TQVhd#7jiM$a>j z#bjUU-Tbh>^Rvh=aj|y2L2qw%{?@lA-t0_ov&u!AZ?h-5OV4}QTskg#!{_Nt-eC_V z{k;wkfyVW2ct9U??t1y5%W(z!@`wR8>Hp}^p8{tU>C>04rysMGi~ULX^~Wg+$dCK6 z0T)iipxv|L?aLC$K+vLJ$|3UBbLYs&8IGT@vL8a*&;_-(44t}s zORUXGI|DU?tn1#4@j6|8_R>N0miv`d5$}w!Q$JE|BJo%j*2RY zx<@66h=PKEhy+DIas~kj26B=t83Y>186=AsktCtXpk$gPG|*&FBqxbY&N(WXCcSE% z;LIp~-+OEQ)_Uu``D4~h_oZ*$x>cvn*=O%_(x&?IXm4%4W8S+|)rKfTupt6b2tIT+ zM*7WF@#%W%DqsQHT>@%yTFi6Xh08=~TzITj;^}B52%xl*#NjI1C1p;yul)T&+K~qu z$aH#az+m*P-=gsD?OZgyZ~>3QVm+lgi!WCG#koUHx)v z%8~D#WUD=)C72TQ7AEIMe&i?=l-)>1tx-7@()|mc|JRbC31PPpiQ=DiY7Av)vfVGS zdi{EupX8IAjEuc*1e=D?37ebE|COYT^0PmaN;Bx}GsPTqaOmfnTahy&$VXreE#@i!M&+J4~%XwtQJK4YG-8OB^ z0F`Vvc4#ogWP!Z{&fY){rnE+JRQk@k?1mqTZee94mghKFG5)m^7OzEX^U>Lv|7X=rxJbeCOds*XJOH)yUD5JyfVUuwMQ7vm%XawQw<7b$>?_E5SD!%EOa<4LFdS^ zHum8WxuD165}QL09MxcWV8#QDRMU_V<=jfSn6DujnOIqaMONXa^+C?7dZxH;ge0w}_e;@3zxk5M*` z|D}VMmP7lZywf`>uHz+)5J5)}#V$l#my;0vn!ZaXvmh#WKvWp|`7?*Kbje%?A=z=q zmYDjv``pX1s%-9p>F=yZ$QhTPPZwB)hB@A*mv?O|3S?tZfn|lU5i_WyRv0W0T<#o< z&8BvHo}= zS~-t}^3?L*p>)!dj?pM*oYNPepFY*(h&V6ps5n-uQ8TMoLfUGWRQXKHs8#gilO7np z%XdIJbDMpJWk!7IfvdT!46&UTaA?}eV{aOJvDcI%EHphrpqt{7i`z<6x4TXj9HH+f zKHyR+%cKEIgsk#Gqj?t_yb8H24zwwvPP~sVbmI^iVJXpJDJn*>$?@FkCQC?wU}n|_ zQHSEHp1VPxn%)v4svqd+Yfgd}r$ix6TlEA_?Iu2aJc3%zpun+_E9a)ZXUM;b3jfo-8LRsk0Ng@A zd9V*xB2Hwr8lz^aJLB5!sws7f85gAceq%Uy1m5!$M#tvQvfT^9vZk$3=JvF-v_0|3 z5e$X3M}?~oja8`->haR6A~$6Z^aaci8-<9L&Jw{Hp0%&t{c-9+l}$6Sv4q3>F9P1U__2xovgo5ti%XnA!+<~eBn}io~YQjQY1Cpxj3?8T!#!P$xx24F_w$F=m zJn1$%%fYhd6~}u)ATu&o9zrWgFu#4z<&_4vo4ey8vu;^3o219#K8|KK#Dxu2Rg!wB zohcH`rt8)u8+yd3yRyjkEmUXgagWLYv3dtwHhb~|hH$G$=48C9k~tEVscu@K&zj^R zmZ}g<>qe9$vcaNuK(OhsV6+csmkAwQvGpX|SZf+~AGV-pudJMtH%o`i5?$m~o!?W+ zddLaiS_=~O%p+iN??LWiMX`AQ~M4}pu)@7$)a+v?fUjN%OtiB_*+LM75ojD>W z)RpUw5b1pX!HUatg{MCaLWUB8sb#a(bgz|-dL9*K9emMMk#8qGi;wTb+erSVIk}yI zp=@o4xDsW{z8&E_VEw7#7X697Oi9tG8-}orrLM-F6x}Gb#)`@jA;+aI;%SM(rYz3Nj0ThAnEphUmAQWT-{D{aI&aQKE)xeyE;t| zwUr$0*|e4F?T2 zDK0kxIw|Th1A3>OPAhDadc)Z%q?U-B&xq%R)$~!^fo{HAx_`f( zUgZLr#;r@Rl!rkug9!`S*_d)I^H*929rbp+cX-%xVxSz5qQ z&hBE>{#grv($h`&nwv8$*GTN>it;-TI@gJ)^Z9m@HfwdA-E_X2TcK`{rCg>{rS?6_ z8eP8T5_8#*AQcU$)zw?__x$|=oqHIQpxV8XA%4Rq5tGrN-_D&HlP&*Urc;;b2?@(a zNvuR7gHf`oQU^qIF!rcGnMEUHd_mP+K~W|fy;sgL@|RSuOPD%fGd|W1Hh>9&aAcme zcpNI?Gp7Aj7k@+!QO){@zvT)4WZ82~6m}=lZ?t1CNM3nO?y=}^_NG+hVI9kBik6of%DIScnuBK1gS@C`>Fe#W!nAZfB8g7s!&d>GpE~z=yr2=@LzE|J+@eCOv)KltQ?I&z zlz~StBIsDqqR%0#HPoY}5GO*{*l0?*Jx&$dz3n2pw4$5%2oHI2v$N1`zJJGe57aHI za-* z+Ii`-rn!19?Tc=%GPQ1&PTew>=yzU6HT=pcGHTpdj)O(4L>fiqmP*ZoCmqU@Vpo^@ z#3ceAH7!}duzGvnVO5f`#YPaDAGIq`mPTnNo>ccSWaw;ggrTq2h)?ra`veVQqAo5z ztdw=H`Or$V$?p~Au#D6I-WVipvg!=~5#ON<$2x;iLQqUaP_A-|M+SCV4eR?iE@xZH z#bb3RXIdI7NFghi+PaDb5c%d0Akx?_b*wkoGx@BcPuIC2ZP`N>6EivEECEs6469nj zuCHyES;s2;7Y~H@VoWqn~T?cRSDlu_KO%f#2U-=1wVnExc!~%tCZ0sKByA0@AAU5NZ*}wJhGHR==ftdE0PUJ7Kuq z#>GZhrRUP5Ajw;+{>1E?Hy=IK8sL}15eRBr(oosHRmi;9TM`kKkpcXXM^=rP^BomI za|crGus?Rgt6ii0S^c1b(F8y;74!dPY4^Z+(b|AH{s9ou@F6P{d7N ztI(3cb7j>ge`AN*drc{`Z~a-i&UR4*v&piml+P1!_SZX)pjY#1RSR~~gNQ~okD}q- z*nGHzp4Ffn+2P43)LJZ0`1aB;E7pq*=m#w7aHvZmuRdFa$Yz)sd&8{0M@VXAu7BS| z-tEWj5&oW3)>|SgDZ2Ih+I|ta%T1_5U)QzTnD&w!=^fJQR)p+ZogqxT6xMo{>{p?e zhhne0_I_yTs+lLTR!Ct()-2_Vb1)7)>5$2Ot@#a=_)um@T24M%xDf@+2fvr3-H%8I z_(Us3k>>U?84QgOZFoi08oh2F@pQil!|i$@dFKV$$ia)JPi1`eaa!^#M|5|X*H`W! zm~}WL2&e_7hM#Hi-MGDWcZLr`X7yu3!}V9HAB^;u%NJ2)fy&zVQq$i~oADZc-g^&a z-fBZ4an{K~rq&xFcn12@3TqpcDJ2e)r~-%1!_}Nnc6|)#8|M;o42>Td@CYY;mR+u^ zt7{U;)lU@jJU@niZK~`t|0r9f{fqYPiY$0;>C}%@o*%&t4HW~KDlQGUI|_}6ffD4$7Tp@xOZ>Cy>hhpv^;lam&TNW>5tv=;dF~ok zPwV}_i61QVmuYBa{KIGH!5QEg`&Fm&UTiyM%;%${R(VgQb5ZK9jJN>FVf@qKop?v@ zgEfhUkxAi#p56Se5i3t!21|hnW+7cyJPP8jo;LlYgQ`?rt!h}zUY#PHKtV>c)p{Vg z`gv0dj3GaO>zPxG63JouH-dbu>S% zlQEbP+9^NV_l(Z4i{V9#1LBe{vuqFFwC-8QvGt^{`XxCP#I-hoU)xF(7q9Y@mL_hk z1iV&dn)7n4osaax$tfljJjF6bY3V`;b;m7Re$wN!8dhlWWmXF{LT&wRfks*TXn#Z5 z>RiskH}H~kNve|m{DR_N2UrcuBkoFzd2Te?rZbAE>BTs_xVQPmQB%{P#;-LI%TMss z&Yt_=UHw5ya>~f2xx=Np_%|jWPKI%>jcAsz>sueWyJyzZmf$vL7Us?=^*S002Q3;r zft4DnA5^;Dkmjb4Ve3rbyj*l1Cfr}F&Ao%3T9YI|_{+{Fs7QC3CtCHt(Hl~2+u*Y?l0 z3B5NbQ!RXwsIIRz7{)JckuhqpVzK#w{8x_n7zzN5Zl0AI@u$%_m3+*R4|1y(+qiIO zN8wslg;I8k*QTW^GrM8)dE`jCv!|A)-7oW1#g(8dwx0fKXd0U0o*({Lj}IZ0@&qC< zJUTi!<3O%!aS|5qDpSp)?J4}|n{MeRqpH61xH=sd_xn56lph1DSK9LHeS!B!kK5mA z!|!Cn)m}Z$p!?Mm<-^2O;~z4TY3O+!j6TmookBVe0RzLx&;N~4(!heL{1D=@$#&+xa@4A?(VtL ze}GYvWahBVLe6Rb{<1)Hdh9^caJGk)4fK&R%F;>zllzv#Y_~fsRk!;Zj43mRF0T5hd0rdJebYav3UY z^gE%U#m*ATa4JGw`_(k8^~wEimiR17xPF6hQdiwj@NmJ>;1iE99b}r(asFrq8f8%YGf5Qg(M<4MS5hO{T8?%sXsuVRwc9t%zII@G(=T_sO1AA5z6U}?M`sXL!EeSm@AjULCmdN-{_a&_1G$_WZ8rh=1t4x^F3YTB?0R9h?oiuq_Mm;eY+o-!!*Q;eSL1;C>P+6} zoeqlN2c5=#UJ6lJ@N@R;3mDCbv*2KQ=lceiu&(*@I97}2`YiL?C$Ua)BxPK}p&L-Q zO_Krt2svHKyTe6RnKy;J3f#9lbK6fCaoD%i(0hLvO0hSELxER`CcbPGpouT{H~a7P z#IU>|q=stD%0~c^hAeRUZ=xKl(mn2|nRzQR^j9t`W_VlB3Ou^5hoi0!?MhSe9RM|Z z-n5z5Syz_tP9G5VHUMdZWwhkosgPTQ!dKCGvjqPA>oye?${`AIfaH@*?Kx#9;Djmw zP!PAgrgP&Bmlc1V@?gFj*`~Wh7hBmzG2_36_C45y8BAmeqsvGLGxx>z07x9K7qODdEX zd$#k}94mFHohK0%X?ul%+r2qT&A7Ro5!20TAuu(YkRm2nv3q58A3-i=& zy;g0pB_@ZNvL>LxF_-uFWw`$e=;X=m^Eq(a@eK%!q{csTGF?1~Gt+o0W+*Lpb7{e- zo`7=5(lSRQ%cC{+gi9{s?zgYHvz=aFyDY;o&y_NHw<^gGX4#3)JvG%B zYR|Tt<^RahzjX8p+~ATv>2kpzfq1xQuT9#3r_7W5ksg}3;bTE-fFqL)chbO2g<_lj zm&-ARB0+Oq=sLUV62^Xn(=z^m(=@8dpSiu#sxY0;5>Sz+e)Hyyxnk$*oWqHu0T&)F zrXc2aq4D7tE+thK49n2UEvTm8cF1XufWK)j<|k7Cn5MU)-u}49N3w#&wxV=1MFq*1 zXgV^Ixy8ObID189bAQ(uDU#?WFb)z6nz{`gM)lWtcs@BTGrx+(b8(5H;rXwOd)dsw@yqazsD-Rb~HTP{uJgnJ4`S;BIwME6X!@bo836f&Z zx9v1$qbqtE`huJ*QDk!WHl{P$n}bBtPTC7I8E3w}N4&AM+0b%vGv)RlbY~6e>eFFb zri*;oa;$i5KOW(KqLCU`-)=8-W3;N-uqk%QZ(DmN8~P3!w+)$rT+Im2w7Ntgtj8o$ z%WJ!AwF+8DU1^cONmOQN_)eXqF$3z{Q@=Z9tP$`2QG9A0>C9a{5yfGh?|*6i zWwdAHf2c3!W<%Yc_Yqd3R~M1qkOgWXs9^JEQqU+jw-^O%EGzz946}x{tEtn%TFJY2 z@9d4j%^{6ry5{;_O0)^k+c@|!TW;(s+eyyZZuV1Bi5^8O#M_pgdO!_QAy)joU)R{` zx%ZeUz5SQ4FjQopaQ`=Z66}P%kzf3w?L?0i*TluXqx0~o<&f!k_uvKL+kkoSZYx!1 zTFb8&c_0I2Ny7B`IeZ0a?AP*MUM#WLX49Z#9?vDr92)JHE|WNG9g%O+)EF|2-|Fz` z(}hrw*Nd)DaI)>t@24Z_E+C-1xzdxA%c8@BT4N~1JEqP`|`HlcxI`Q z%0x#8^wM3FZD-}HgD`6(;fTN}bC_LZI7I5js_^dG=dt5M@!K%GQYE2N#Wfl(DHG0{Giz#|G z6)8R*yPISoA@;8QgQbNw;a7a0H>21^BTSe{WvzLcm8ebI;@^arLmyBUj!qA*Rx%12 z+^|}+EWEpPo_xLgltu;U2+~NlD9ZccljMs)=q^4)`gNIBb4FL%HO=K`^96+4W621g z;R@Lc?Bkb`^h%2?BB+SGZL9V-y0*>9hwAL-Y@bA_nQ4#$zpnLEg*1-JD4%Q}ALw!6 zcYzJIWf?~kbcQ@*K%Wb41ZaLhHYX_Cy5KYwtOY^%MT?hOY=bf=tKK^jpoN1u+6Ss^MHw2soLfr*&oz~h>=P0Os^aO=*d%4xJIdfDn2B541O*Ouf=_#HyEoYkh zb5mD*%{6+n9oXSP=>+}@lZABn$&%Y5iQBY2K8^eu?ED|5QkY4lhX*!|_xs3HeBUtL zKrNJaS4e9dxR=bO*lk*>cwVE63p3+ph$`5;!;Z*|Zz4F}&C$q>JlL0k71U%_l{8v{?u10S_`j?CAiz zYWu#U{am%h*S-n*`?GqYks?Eb0-(1HH4zaL&KembB3d zCasJQSMAB^=UT&iOe#Hau8oU%l*UJqC(S=d`a?shDO-(_bz5Abx@;maT)oFtkJ+S*-GAm#xJ80RDADs!Fh4%Y&D-=dv>!K6Pbpfa zIOPgnq>rZ9)dUjT^+qC&t~OTMBlows6W316^X$w`RpBZeVgykkk48pdiWr28M`V2m z{5%E>{l%EpAF4b6bl%14QIe9&pmpWk{c5v&s@usCbA=F=((TJ~)GN!esTF(3<~1rM zcOR7Cez@!QyFoWx>g+={9Vzu&9mb2DlUb$@AfGJdyeE;emluP!$C=y92#{Nt)I9XUbX7-NYGl?HsH?C-CB_Rti>7 zs|_d`sf1>h^qsixG>BhT#YDUo!9T&qoKz~N8O#63QvMi7;z_Q;8S79JcMx6vhE-r ze_xZ|$V*Y0Zy!9PNWQbzw$?f24Vf{7Ead4n`gfec_iu|7)M%9_^qI>n(X_7a&m4`a!nP)vV2fDYN%~E3Lb*z8U{B9zSI;P(^^Cm6EZ;>d|e)=r8D?OQ+4b4P)WL$n>Q}&kIh6j zsh#I06TgQy6zi}X?fIo+CpXq5Sl(iSbpLfdK!S!b`G-0gf)CaA`FT8@i*Z)}*ks?? zSUOdh8o;rCpW=wH8f1D+dL-L=czNbZHvcD9t%BljgaM0jmXdc5NL*bzLBsTdXLqL> z%=qdk4?ZQK;~!~|tRE<_Hsw3ntCA=14{{I&HkTe6aDtFa3HyfiV8!lD%nUWbUPaNJ zA0kJLm1hkUYZAr&O)uKVGrn)&h++=)97&Rk%BMftqpswf6$M)RFxiM?$BcZUOQf*g zL!`8m6&pai&bFVYpH&+WoFFEMHjYofsZRP6Kl=wv2_3AD0RYd9k#PloY zv+Ap?Zb0UNhj$)N#cZ*G{7ucZ&~`d)%^x20U$gY*|N1bK7BBd80Ip5ctzMX>#$jK+ z(rFTZ&OJXWQ00B&FbSoSE_OxwX7u&aKoRHVev`Hl(Zscx@P~;0WX@*~$}?@3#ZCaU zQcsOz9nWPR!>eu~SY z;*q;Pc)%PS8`d-J&dX{fEGL)0U0lakf|xS$mpv>- zI5-4$vSd>$P5@}XQigLqh3Ez$qFSQTd~8OxR)mn-o!UF4csKNGLQs@9MH$i1lA}k2 z;mjc2JvM8fr=3tEXzwR!0&9h^n~MfB^Wz(*`O}o+H*ME9Z0(;#5vLS5*~(fFUB`us z-L7o5TVKA-22S;JS=G}{mcCZ=X@@DYSM29I>4G{nAc}8*1fhwUdqx%zn@pBptTw&7 z^VOclD?u8Wt|yC5;w-epa}Ou69)p4Ykp->}KW)6@-vRctB6n(gujf^i3LuCH*p59; zPS&yQ0oBZ5Em1-$4~&d}*jhNpgXm|$$v-|5lfVPNs8r@NU!CTDuN#)Re#%el(r*@4 zzAqY(D=ETN$E?*UY&?PLg{WZBP^o(WXIJX41t)=5P zVCByM=^tZ21EOPeZ+&%t1n9y<`1YkU+$QDqc*RLs<3Y~Ko5=++Pm`@x5IXdxWV$)G zw{ZeF)BFs-(QE;kqp7&ExIxIMmS3>HHrSr3cl#rTO78y#pu<`6YYGzeVt%ckuzCql zaC-C-+?~C}Rvr!v3CjZI0gGfHJyTkdy~`p25jhE6LhfEJ*G|NM1XpeM@=(T$0;^$# zj~^Wxz$#^7SD;#5pmoU^>P&G^%uwHixGsIYlH*bKi4wsdaY#t;2H_aqfzQAWn9NW7y{@P_~(OyRgb{I|XN z2{SI7DsuL#%CHO|v`xg0L%iVssdz#1IutujU&hq~u0MDGRJ8)Tu z85x>yzR86awatSlZE2+N4p9)*8;P?q4N>r*Mt=?Z=HOd3DdOvV=FX6pu^I=q#tNVc zOw@V>2h)E+jK7U*^_{T?O?11rT7z|pCXS4^TEwYLsJ-?p>o=F?@u%KK(-pfmOOT@) zuJ>m?@lXt+G+CG%TUeHlPQmjBAmU$owS`Qy@2!P<@bTkElNRC{li8JJuJ=dx%(_ri zaJfw07-fxu?<_AJUnM+B|C^6E^qvz?8$j%4+BL**1Q!42a<%C;2&=q7!JRS%`Z=@R z7FUzJ1^7D{zb0s<%97g_!$!ia4y6x>aow2rR4Q^5m`A$2y#_3yu)DMS1+>UdFq6D& zcWovD^dzJOh-)4rvU!5QZ4O)dToTAl!b2Pvk{RKP?G@`!cJVW!>b7N{oerOB z7ezE&<{b$Od!6d1$F@za>n8Jcf(}UQM?w`v4n9|8NH12~mK9N&H0IalgDmcPY_`@& zwp>VS%Y@aS3f87Vedx?dB#m$N9Ix06>3??o-=-kg6I;YA?ixSoO^R+To0}oB(G>}T z#NGNe$;rfAW=#%_@bk{tQM<-u>4_AEiooND47*s}tr>!re-cwZR3JvEvl_Q)3v_u{SlyvXut7p4~ zURHziZyIW(Zhy_vNb|`^^Pba02sq3@cXQczZ&mK%&qkU!@>?Ay;5~INPkX&IP;uup zS)kH~xmCF`Oq|oQ4<-yXsD+&O83o5+K_txj+_Q!xOfm4`pbYl*H=jS>>o2h=<5A_d z$xZ@pp7DPXz1>Z33gZvuDMV31LHi}kge8bOWEpDkoSUe=6J6c<-s)ATtccyX%_hvQ z`pSy36qDC{%7rp&#MnW8P0VmeOT=Y!SA&C=8daJKw@s~uJ?@U4XIt*5S9p*X!Ru}| zFLFA|kiHP<=^u1(CJRF3L79^lu>f!g_)zj^sm`Am6m+id!%6W%33H0Q>U5PtAFSt% z87k6iqwOLxUB)fxc_GZEY*GB)kZ7*$GZlBUN8paVu^IG2TwKb_($Rut_e?+i!KvGZ zP?-Tv-DxYf*aB_TfHq`)NH&74v7uhY)J01d#T1hnui>3_hutO9w0faBNsqKxZy42? z0eWq5+pMbXMQkmv2WP-c?`Q$h$n;ai1OXGLE0RH!Dj03&kcGKV^`a3n)3DU*-#58veCH_dJLwMUn+cPJDvObXBF_%$q zRtr%Nr12u1bIr%w*-RlStjn;V^~T1Wo7+!wMTVLWQsRmzIPHVGbJR4Jcl-aNpNn|; zgMRM85Bj;D5CoR-$KpmV^A_3=dq#1bE3{c`*KVh%>GWrXXsj&09E4Nk) zyRd0a^l;UqZrDvkW>x?IQ5J`MR5PyBBPU+X?Z5aT3gCyHJY`AGMi1qlRe7nZjN(<~ zW`W)jlQlu0d$s?<0fbot#*(I(zy-W>KJRhM^$s)7Pqk_srWFj?dVB&rJaRay>kVha zVa0W@`Yw?Dk{9-FH$8ncLyawQG(%Rs{C6Pmr$hW#c5)0w^tpNBq>zOwqVc7X!I5N3 z2tpR>zH$q;eF})PUkt?9hg&de>2EV@=KG(*$>GtXXf+g&5+4nh1tokP>*j3H@S$d%a( zBpxEEE1dJwcsWbgc-J-B1($m!F?MGLH@O_QE`a7CB^&Mr!5MH+kgMvTL{RBfw)deS zwEip}9`A=%Wu{Y^0DcNlfVouY!qhTXU{v7@TCBA?>U`Gwty6KTvNsZh-PNfmQN>Xu zZu_6?dg|$RHxXx<9+!9bth|jDAA*SsNlx1Fme##$es^HuVPt(QNmnNbs6s#Y(LM7; zVGrnczY5h`*zDE5)SxlyVF=|Z9O%yW%sfNRqtolXTf(B9*T)hAixBORlqR9E@OD|S zl%?c|<#R34y1Mn!-_`tSW;{WvXWpTT%2CI=q}Iwr))jNsv~HFf<3h63E)h9gFR#Ag z!3ysR$80^ftu`SonGgHVy1z^No>oD)Q0ze0$8M!n$nPSW8lNjY3E^^^;)V!}NcO}> z3w};Bt@p~^B4&O2q>48a$%!$-WKKGtGb|)u2B62fmbq?jF$-iCgPlopa~tL0{>?Y5 zOD-dxUD($o0pkrj%Bgxl&`ZVr)l74QB537oyd~V@UOdv9o)MLi=^ObDmi|q><2^v+K&r7=7v#_#jdJa=iQAp3-s4TC&v+yjCRICxNdJzHi6nTom ztLBwXi_}~XrVN}Yud-JJmtOvAScq*mO<{fEAP&@r2-0*b4m%AKq72Ghw}>^Uo8vt> z>wL^9{8&ul)FbvuH}CdCNK}{Wx~Z$TcqVcedaEbUo62>qnH#VFsMAK7W~!n8Iyde%=GWi?nrN(&bo0crgb zcLnZz=DM=4Pv&8T&PHdkTTzBcNVjG4KafW`2a!Zb(!z;FtAN#etBDz(>eJXVGMUi` z4d$#m>@lrGa7)*J$>_Vr)!(({or2!B;Wp85-?D7o3!F*8356BcdTZ+Wv{E5 zaZy-ZUkR@06nvi=1Os;gVa-Csjd3VKkIH5Zr~!u96+QJNg51 ze3!HtY_7lDwOhD+4*}VYSMt$Wt;rGHE*(g{uSBB%dUQ74suMK~RE;gBn$b<(CUcr) zK+Y&X)>9GK^|csb7u^<-`}o@Bmo8evNmSet!Z(WlO*jABvj_i`AMv>BZMr%Tctp}) zhpqe0-$4>?eOb|8C5M@E{t8j_jjF_qrV7W@nIn%qB%s{cAlcstH{CQuCa6g!6b9tJ zO=s&`?bVNB+jhDvFtbN_gEA;rBo{*E?yo1W`;G7RpQGi6LHWpC0lWg0@{=cz*JS}` zSVQG;^;}PPip_B@0s5yKE`TH(cddoNx#M;-I;ghg1w{k@zdB5seRF5)O-f=FIn>b6 zTh*$J)TNOMStyJCbu@G20aynAN|vUnYy zvB}4{JKxu&k0ZWLD0vct;^hb}-bESUQY0Jg?M-#2$ga(9iKaYyMZ|O>8uVR3;Q_Pf z>l)li*POiP^YGX@FyEfw5%}Hexn>mxnGa8%nV_22lO+NK^JheDq=C%qmA`-3IvHDW z-Gv0(&&lg|r|qWS9J`Uf?Wi2pF+kx@xGbzkK>w*3pFJQ#-+NDUpm3Q6-PhtZ8Jo=g zx#=rY{tbrl47)mO;UZ+mxqE-?$%tyTQ6r#5NSta3S0QbUWLT_1$yK5t6`-=i9kbyz zDq@ziz_q)Lb6R?$<1TzChpInExJ(DjQSt6{D|3f6T-xY#E$I8?eEl(iLJ{VXC5G5< zKCzx+LBNVj3~$Ii_*Zt|Uy5PNd`|d>JjRc$C+ii@L%a+yjVJ#U6aV}x$*+ycDj|*& zPlJf^(fKNToz3bw&@OaZ$3DNHtSrfLXOm(6k)kP@#srvF%Ai)xPjFGL<|ULHCyJ13 zRkVd*Npvie$rS_J7XQRl(nW6S>je3>KM^L_ycof(2Ga722h2I&t$B!$jdGY*nyeyv0DZYEc2^2uV^)Eqr zK_-Wpwy?gw`~AQVk|h5Nl`VT`Q+ zey6#2!1>$0*0wa~R^Ae*9e3=0n`0k2c3V_8UX_t?UAq3I)4#i(h8gea)8{*vbJ`d< z7DAuP{rkQA`|lgfG2ek+pe;SSm9qH*#9ZI_RL8yChAm70D}_18j4Lhiyz7j3a~7{E(Hb`4Hx{_NWyKdVjyD=C4+_*W$CKL=mEc}>hd7!Y!PJoaD%Tjxnd-Oo>T^=6d80T5m)!dY1v`O%B+m(uM!Y@p*SFy5(ZF`X;@!)XRw3I{6T3_LeXTCUs6DHoe5B zryr+AzD;if6}|}RtAMt4riJh~%kLY5I0ujiB?C_}D)XolP>Ng=e@zv<@VH)R-V=IRz`g;-A0hKHnsor~u20VQhoq0khBZxr;T1tV zN^pbPc*HxBvd_C(L+$@;t&=AgipvA7g3tP9h`5Z%l?2=0icjDW2knpacD)m_!grox zq0^dZU2ld`1r$v^LNmePvvP!~77M?#1Sc>taB=|eLD$_`Cc?WQG&1sm{oK9s`nUi0 zj&t-@?G>x=O;q>ti-H=<6_78e4rN>=iYkx@)Y^@ z5Tu z7iuWqdFo3a#{gLOLD;|B$uzbpm>R+ym&i1fKp_|zAl;Eth`)IS{FCSvX@_RgeZ0+1 zc*XQCL1==<%9?$d2I_IaZ!HdX4vH#?5CATC&$pCJK~>Qn<`KY1kC1-!sQ3cYJt?9f zdKu7BLjEfVtFz3hX4A`E=?1L$%;er(4B~6_FZi<-fIV9*-q=4B*7mutV#=Pwc_3}E zJtVBDP;Qv!Pk4MDh%wR1W8MhGSkxB6`;;Y7!C{gk~kD(<`kMMXoK{{y$g&n4= zguoJ(RI3sCi*O&}5$_kon#$|bkg>%_ym+x08u?K?<>!y-QyplPFT0a z4p|ri@%PPu7zgI<=;9KK8?r{=dCEwwN3Qzha^hP~WuCOIR9#x<0NPg#zZg;Squt0| zL>}{pR)u;Oa!92v=JW)qRD+9vb*1i-Jtw^DtN!yCZwha^$lcJq_WIV57s_CZRA-sJisfTML>g z*y3?gzdwjNX@eOtljg~9Sp9ZiVWfe>?MT$!1(QKije%f4BC65z&C|=Fwh2!#$Io3* zQYty~=xXB0se9+{;wWii>tc(ay9=eIdGhGW!b0id0yWe`_=LuJ%*5+CR~tx(o?Jm* zn~L2pGmmZDXxWg{P;^n;Zb679MCG_?RG=!6?83WFYGlkQmoYH0Po1L?dwk+&|KY?; z!=Z(^lQwzGHQ1*vps&gbvp@Et?rdQ8hY8{WiGOu7^q=3$0$1fJH=RHBA_OL8l9Q05 zpu*2b{e2u_dElyA8Ai!tFB(35%D;E{^ZbpUJ^o)0R12;m{r@LR9{-HR#a^SM5J#U5 zDzj@u$G#*R%(G|Zg`EYYR|V4QCy4NVxyZh~_cBac$w2?Yk`*iS;d9wBe6{ZlOPIwS zZgpJKH7Ibdmv8^|A_afnaP4z2{*%C!yyx2UTMbv={B$w@@iu)vT%+L-jFjjqId(*! zlf=}lQ-myfdpWm<8S3=hmnap5H;p}3_Yj}%RC&`Ze_j&a4gl?lh3MrjQsc*a(zBKB z?B6f4Gz*FgmpX3Eo$yh{hS}H7MX<$??X`@sdpWHN=WX^5eoAYvy0N_Y@;vRtM|`$` zuaot)`?#yGDT_uK*{Y@nl-ur59=BJ)KD>~=Lz1-D;^pd3Da6($uuc25;ppJXO6lIL z-F&USpXc_7D1^_3Bp@KbZpD>4fFa)Prj~**b71WH%)^Hd{d;=@aD-TWkDELUOigSi zui80Z!eMI3ip@>4dpz@fIb}ebh?HJz!&E}nmXgtDe=|1>pU;66wvzX-Hus{H=t5uKhB(TcuhI{y@pS%tDIu2Ku+d2F^ zOUmw-nwlEPWhTR-S$b`&e{XsoPk47DR5-q`7?1dYM_Q(InnKiok-)18eA~t0v@{)v zaK=VlUA9e9yo~Llbjmglq+;(v)3mx&s;waWBVWa|?V^0z>`jPp*+zI%Hv7Ak-Imgr z3!xvgFQ)8swX6yI4Nt0Psd@=m?^4w+aWYF}xRYBC%cEordn4|(?+&>+&$Bw2G{YWh z<|QJou7OzZ6K19ciPwP!9fw=4g^7Nd~kI-*&MY@@#LP+$mwG z*EEf%;P-UWyX-X=d%v)ht~Xn|Ytr@_z9zEPavYBD&cdAMf?-{;w-MHy#SmWERlI4{ zt%7F_9x1*q%O&A<$3ks&{eW{4c))n&`vGrM|5tYg%n{?g637RgU50r~ZlEU<# zYgS6;xexh;H?h~VjwVG4Lu#)(vDajmN$WGVpokU(613adlz|}$R(m_)(-G$~#9K7% zyjP$v{kY)Z>jt-C?L4(0r*+k2z1=56$=Y%Y>%agC{CUaI@xL)Z1=a7L8|Gy= zsUX&*-DD1|Kh~_Ze+yheWsX2 z_=|E@)fuF1=AeQn@UpWhu*Gq!hp!wrjx+Q=0TiXU;-TxZofcPtm3kkCf9%0j$N-LB zccVnfKaAVYFAmcLEgxInM$;ZQ=(k(g*b){tavLv?TlGF%_^14#DPGFo9!lXs!t6bgS-e(6I;8bvT%as9U`JJJ@XkS0-{6sa^r<&hVI!fa$NtaPcY_ z<{jg=aS$_1X^xEp8|`TI_=6C&H`vm`$J}i}(V#E6v5SARciF6!`@!y7QT08W>62G; zT_FxE9?s)pmrHYk8nbuC_nN&g#P1OvjP@l`L?5M{_k>ogjvIPHrBClqe9+U+7cW!_ zxGjB70FQ}5Vf7|rbv;)iVPX8@P<{{*dJiokdUq5TyxFQ22cyvX_a*5H)PVB10Y z{3t`M$>>}ra;?N90qY;PHd2okGb=2YfR}N8yDfA$K+~DqA)ANWI;F%lz@Ay2#+}uH zwM~IjMzMbBLTN7s=~-f8`EROwSvzx%l~ANh?P~d*~E2M&ZCcp zJL+#meL7OFWX^oTyTBMcbh5~GrpU=@aOQyxV(&HX>st->6hlS%6}o2HE$3gDlUw&o z#%nsmlTRYP%~+I0iwyDgY$)=fGe4xiHRRGzK6==YoN!28HV=QUl64B6ajr>@3BRoQ z#+162EEuoYW3e)^JA2Pf!P*Ryr75wp-dpwvWw!FHe`uFKBjh^Qp9xNrwj=8nfB23#1v)>}h;@iW?IDnt_ z`M~@+L3_URf#a7pd{c>^%1v5htR1SK)9~ZY{Vz-0b{(~|_4y{U zXV3lI*8Sn*n%eKL=KsMiJ$L=u=NoPt#qAfG=A zKwd|5V&@~dkl$=g_)ID(2BxpRL3esb9xTPy!}b7Gh;#nAeDqCEl@CKYbp7i4Y?^a!rc=PRpqXG=Q*^5;El>COb{Z}1%=erqICU@!D zBRuxsX}!_@N}Hg=xNz%dYt%>UT;Hszd-rXA^GZTyZFTWxKN8+iMapYpZ+ZSQaq4;% zurG^M83x%*#%Ncnt9tG)M=HKd%duwxO68jC zi_(W2gii=g${fTU+xCK6EWhiwpC&TM{G$Gx{;*o9Fzm|Fh<3L0?+$= zR^|cW%=n|LhzcFVtl`-_YpZegd&}y2w_{EqmbiVs*+Bju%=Y&c9#vA8#pBay?iv6= zpOAl>Pw;p{G7jsPropkbr8T4V&4hLibpvqz^Xcvbk|bYeHKx$4`DO9$rhoXmkS?#h-!?>Q?S zd+6q`jQ(Hjy?0oX>#{ezmIVuf1rd~HLqr5bnsgKa0Rg2-4MjQuA@ml}B`8&?(p7pF zDWN7R0#ZW_kOU$G2oRFcLLi~M51zC4S+2b|`#b-A@AdBgxWZHJXJ+o1-;{esfQ!7sv4t zQKjvj?l{50R#gu3GZ@!O?Z67amo`bKRt(6l9dgb-gJlhBDaOyj5JGv{E-{CA*S&+b z%%rD-lJYLM@j}eGH(I6D^#>N$$0$%P(E6n6e$^oltmzFGN>`5@`5Yn3*E(qD5Uh2l zVM}RX>b=ykqQ~0CQhzeJPp5w=Xl2;#Wdn3$C1&ff?OGpP1gG@|yH^Ih*3=W`;nC+=GWkdbuT68^87qi?MQ=5;3PW!e7ocA4aa9hn%FNH7a zv9{Ur*A)$;@#gxZoy^ilumgSdD5=B<$;z@s#o<#Ol*_gc5*e2lxWy$dyci3j*3=9=Gr41|OEm4rnLa6oRhORWimTA#MOD{$ z3s+oSMQbgi`G7B}reVYZ2`J)!umqHB8jqilb45S`6oGih-8TO%QhVE_BhPA-Qp2QM zPlofbb2V)Ko*5K6Z(O#c7R4hjB1|8#g@ViD6cID;hFjW{)(S@k(dS|}=Z;rcH-|ND zKF%+fvaHAOR=BqG`^}`}_tyglW#&+7)oji!r1+iHlywXnG#ZDcb^0|ES#^a)H?jT| zIQXG&yDIzf{p7}vmstabTx`2jW7`aeLn_b)JZ;4XD`R@5q!fd<2{y}9f)$43<5g>I z57JzYPjBC_sQZ+=wF===9+6tZ+WN@D*6U)b9@~C!+)35#srr1T?csT~1oyD%tV&7M;*6vyv2V;RZ7T$HD{9aLrgi!z}_^RtzBv zDzZ%<$-;j&hRgQcB4vAYdv8#Pr7jyp(+W3Kb$2=Px;?6T0a!p8{}zc7SR-8X_H@<; zL2Gg@KWo7cBvt8Oj6F1_)Jw#@%_yA*g8?E0m<>DImGQe#t7?Y6niDroOT%d{iJ30J zO>5|;d9Z~G1-1(n4fJ;%<~H6y3HTb73u62%Bhl9qx{gBvzpEwu=VMg!jSOqs!^b`YZh7ey@JFb!OaAppaJ#gpKRv|Dotw5i}C87;16 z^_Z6!Pvg}s3<3(v9t#~WI0$>r8svT5vhfB@z2%TTW-2zOi5B3mxl$vIxP&SP@$qeb zZ=c(y4u>ePj_erLkv4af3-fG+-&R+Z+|!jwLB1N!(@tAH9ELV;Sn0t`_UP;zwg`rh z2O601NT~m1iuZQ?F!!l{705D8cWDHzkGh9fduc3CK%ji8Lqz!~3G zccRccpo>l`uT{q1%C}5}a~m$YcRA_j`)p?EH#Ir@7aQX?9x(ruy=Q>%q%@@sSC3l; zQnxcpN%yyaV1qLCz_o3j>~{7)5L;uLipZ;VrmNc zb(sm1%9Z7|{vtf5 zu22e2l+Y^6(^Opb?2?;NTXwhgjQlVygjPIlYaz4=s8@jg%g!!=rvZ++aCFWjB>-wI z1za~hC~+PgLdal{cf9YX+?3pUn(AG;SZ`F4=3NTTKH85IhmADn(r+<=MKeH zb~kKy_z~q;wNf?t+BaY&u7MyF2KScQxT{T@N?qEg0Qn9u!#IDrepZf%p7Ab)4O=Rd zyR_d51e}j|=zaV7e(%ay!r%g2qrO{IZ%lKQYan+*apN57h?SR$!ak!^ey0=*pHX3s zd;aGwJwD&A80HOZ^AOdc2p0U7Q?H@WTuOj#Rx!sY5>R%l;rR2s7bUs=1g`BA2VwoslU z+5|x=$pzPx)5%+?MyT^5+|Bs?zT??+!0U9Vql1dhu~L$-H}tfwZ_-- zP#rEAHm!IoRldXptfsIff-23gCpoErEjpcnJJJMAuuPcxq{F8fnZg_MS;=I$`sFYE zQ)2dAZ=XSJS=Xvd`|Smvgo3E`W&HX2!muw@HHOVy^)>z$LW`fY;tw(EQJA_;ApE5) zxKHvVXK)(^#~!P+J#2o%X@x_B19CH70Gx704&SBUU*FC(5u3T8J0~p5p+4Vo7&WcYq*U6+{nWX%WMb>c_IT;dZ|GqT`n{ zrQJq2qcipqFF{rW{Y%Um%G?O1{X2RDGd*el)oGMb5dlY)#TD5}y+u_&W->s7Z7Y`WZq1Su#de5oy?diN~<4Cz{Z7F`O|7CD?^9_@$k9(EuF zZY~@9=6ctp))~?_zv;*3BF)@nvS(}J%zk97kL$~IC~d>q`YG=@<$SE%nK6Cu_i+t9 zk(>Se9A8UI=Z^HmiKk+h@oPJDjr*Arf-LL8Ah<*xSf(`F7w};rB^IzeTf`aZBI~sp z7M8=%C?EH7wHx9%Ta6m%~02R@MFLQO2eLCCENlBETCIvytMqNu} zzu6<*_OTB#{tK197~fG~m}>mPSctGp_;wd8CuDiHv$WIzi6DA_i^E2(%!cFo%EK^L zW*EAy&v)W+U65DS#)7{GO%%!t+LYgd*gZ&`@;Kh?vwGbx7>%|%aVoK%X3Q%0*>pFdUV9pfEk0_1a(D>#Vmbe1LKaH zL|STO6_2-B6UM+$^^hZbOSy=7muZ9x#Xx`ftW9{f%*JN=glv zwtlGvs&Vd; z1f-?DW+M?5>~f>rWejO_6ae;g(FfI^XY;h1>&KkEhb??!=LQ^FX>h34cuHds&Ugln zpZUYDT?CA~fR=&ilTkVueA2 zw574B4H*u$*!7_W<8mLez6|V8zi{2D8QO8B(|p@M+VE+ho!e1o6=dBSoh$`8bBG2UfJ~bLhiian`z8{<{Z`dxX3K9mqv*&1ninio9UsEAq);<{HE zU@^=TP&PzF8@;MqgCP{o(KeSkJi7WQuPEcod_Oa;!m#mlI5z;ni}eUJ*OvNe*z1xC z6<-S>R9AhHl5T`I01iUH{N)W(a$j{i$0lsigJ(Uu8---yY~%Gwh-==K@ql`CzW{}^ zw$NN!eo|jv&%-4!@++|^yi&NPUEQF=vz(%R;bPWeH%ezLFxyx+;F zoQPINRtH#Mm^=Zbx%?COJ-4R2G_i2i3Pf%Ad^S?(rC zm_TzG_}H4au-I%sk3!vQVp4T9pcnJ*XQD|sL>XWf$dx6WFmm;>epfZQ+yjt7ZjCkI z&y}18J)9t_8Xg^Rt#7C_|7ONO7`h&{fyKZY0FM@J7XT&CqXOr!=v&{maoN3r6=iNO zI`}i6Pq_D$5il5e!N+fMqjQ2GrP_7H7%LE_KC`?ImG&8Ee&&|uGVSgS&hB-QDU{>{ zoaIWG`qR%3PQSh7p?BT#Eg##$!pDH2h@J+2j3gIlH!*kb`l5%zib>=RIF&parCw zO#5S0_KMtmetP6^6f0*F(3yXakn{s#Am!$)A;-Gs+V@}Sb+Gap$U^~D0o)&1W%vHl z195NdPBmPo_BmLswr~u8D|ncFP+g1NDE80f`0EFTh8(#J6nEICd=t?MmY^C|*lk2` zN_*Ui;*u5#VeSLkZDgMND2cP#ewGRy)&H&DgbY|BhY&-RQX*VkY#cS@blU`)9HB_NUf?o zV$u~XZ|J;SHl59$egbJrvduEkxW?|NLQm) z7iagbbDKWlcuoD%kTd@2e%2^prd!*7puR=OxTFAO%@e+-7|U{SN;xgBY7Z%f?_Y!3 zr9b&9SYb0Aaf^nnj8NPp!N%CNwY7m@iU{CF*Otx(9Nn{M3P?v9m0t_;n0{(5R*%<_ zYwMAtoa2&m<=mwAyxwyizjG||WbaozAp1A+{8NEc&OOL9714J`z3BrQOWC-inFGC~Z&GE_^C(h#vfMi0_nmo^<|noZ0gDdZK&V ztXOtKV*H-#{w?(Gd5D$Q&VI|YzT31NwPoyv2h|W1qp!R1fROV6CZL;n3PQ+yn*?2- z2%k*QkBtOv(SeC5lu<_O?+gN;OCN+8|1Ey}ENq+}?Fa@(Pc(~!q1TF9t2%JaPyE_Jep5t zr7Gr!=mWQe)%J#bDX~dbiDK(byafwktXd zsdia=iOcW%3YGN>SvAf})CIdO&kA2n4f35unYr`)bmTnRz)l-S@UVMX5RR95vB0d% zTx7P|G*ovs2P^Co-}Y4%{CmGwerNevar+!W7gX6?oNe;nuLJMnw-7_eBp2omU_s!}k?X373zd z5uO!g@Kr)^a;fsIF+5yy3m>)WoBSSPguMrN5PQ%O_*bvS&&r4$k1_lF!JBSyf&wKv zUT0GoJqhp$NaN~X{=IiVk!Y3ss(_RD9?T>E?s=lXL+zj?GF z6oAqvJ{Zo&SVFSeVk$L3V_!ZHgmlUfM{X+1FXt=eU_$oP%QsdJ7?qnoT+Cs3w>W+= zlC#4G5T)@5N~18xiJwxL4>X2l?0WD#xBGI|!OYQ7k74aW&mlug2iR~*@Qcd6R?h1p?#OH z$_A}3IyyPM0?J|e`R=LiEjS4mdwY91v(=(wF8i1`KdC}$DaQ+`4wv{pd{C8eQg3hY z5W;+!z%!1_OmjuD_L{kwtH@p2kZd!EV?0gc+Zd659~_W*YucwR%G>e zAy+m8;9~LDh4Xu^{a6m*k15Z~N&WeSOb@J93;LGgfMCbW*~*Cd76&DWNf4=(@+~Po z|BNBLsrREbi$`zhp|%hP?+L(}cGg|H|LeyWUF$U%%9i;x|2`P?`GWhci;7P=!fhu$ zyzuC7f-n|_W8@Z2v0b_U(gWYxh6DM*k;@vPog#IzK7s98wj&>{oi=t0QIChLkFBF% z2vdWt?bB%gd<#5Fxzfe$#qjXSWJGEM*dN ztqqVcy?6Z2F4hCdPz8{b-1r*Gw^a(`%Au9Cso=JaPR8$SFW(zq!ufqmvsqgRw#Cz_ zOL)w-0#Ioy;J?wXX=fPq*SZQafK7oSTAi;A4=0&B-=P+`>oOrymnenI^QXaKILHBkx;l)0qJ?^ z+eotT_z_wjBez4SxwT#$-U`IezUe}J)Pr_#a5hR%8}`=1Yw0|1AG8NyebqlJ5*f0s zKL-c>A~8240RJm|rdH17&zsqc?QPRA{7q1-qbCNli{MtB#3z?U&Dl6Y$c=AzKD*0y z@TD{y6>rz=$N&n^fNG-U^`R!3*OUNP`N&S<+;*gk<-*~JD@TTV8ZNT>H*?`r4(jG@ zfa*8vM*!yblct7IW?;5JQk`$m;w6DLR=I`Ka->=<@}s_!oMUpsZM!*0pk!f0jw4-r z8PickCF~Yk1I6vOt*bLwFjOFq6ay6B7~2zKY)5{<^P%qre;C!PK(HB-e-@4UBk932 zbxt2w${L{O)cF{#wUlj8K7&sNkN3BOuHyMx&0-XxoK9{vHlaX2h7Dx@2ScDnUzc_pLMtK4sSMHW&E83>;p;Fx^!&+r<+Z8 zyqd#U`bI`L8xT`=P63{(4xZD5B13R!zfa67HrPFB5Ty6Z+)5=<5tBKz6hsL|Xu{$t z`^NwmL9SnE^-Cq5dWi2FgIo!cr+Q&%QREsk%u?K)2J$B})3ph6E6F2WS7Mj&t@+6I z^03wgXlo6q3ElVdSC5@DZeJNEPh2xO=TG{BGh$z7l5nJO&B(>FCVHM9xlXmBnN=?L z7!A9!vg*IRAMwhVd$DMJlK?*r?HTJ|<8P=rCm3IT;wb_dl91xsCA!&%_tuZ@ zk9nHX6|$8O#GXgh^p;yI%~>yjSob&ntS!C!oeIbK6h9E(A0_Wh_tjvQ$)8rr-UR8+ z53Rr-eJnZlXQ@2Fxw&st$ysh480Gd0S8_kU$YMN}Ut7->3s`+S;?b5d0BK}oX0;rL zt~PlwJ_#vuLA!#H4*L4~jb60Nvob$-+h(U^a_&aj2a;uVv2l4s(;8OJ*V{?|x@px7 zIPJN6{)PoKc&P)~R!(UWNNyBd^Bc8vdf?Ta%7I4{%kCj`S6|p^O-7ifRJ#nTK|~y7 zg99J|-t!wALGnrEk9aknQ1g`#m$noK6xQDI=wES9(ZVdoHdoDL)&Dj2r}^vV+X^S2)!eE%7+nH3}y zrX^li=}kjGWi{rkgoH$^R2>GVxHO0#TSl|RVU}uYYSd0)ug|>9bub z)X#6JH-9Q|87|q)kJ=vKRFwIv85S;6n}kZ%;#^f;%%0` zsveZ93NS=MeXp<|P-Fho{+BKEr~L~Ux_3zj=xm(cLSv@qG=eYe1+m}lmfEvh4_*D` zt-shx0F6`wC&o;*vfqpT`kG&iHBe-87r;;#OV07{`NE{#Qx;AVt1teJa{kF5p%FkR z(C81t>^)yN7zel`!qH*yzpnR}yB?eb@L0r0=Y)Uqp1*%}sOfIFI=8Qvei9(W;bOHvDYm~-;qZw9et@Q?=UDzDT>p0q*So^>45z=8 zvGQo^j@x(!(e*jnx@Jr&01fumcZ+?g0k;k;OlVt1=Pg^0!_9TkKac?AgX3e--Tdw0nyS5UU(-$oTB)YL}rxyKnowYbUZfrhi8VW_jGW9g#FP3AU^Xy zm4DF8#NqtQzsUyEyVm=l+h$n5mBtZo!q{X)C!vMQ($_DFQ}O0GE-pNFMe&@jJk_r% zJ-~=thS1#yxVb=S>I%Bz-@H-pV24l3EKsN^7Q$%Q7^7E^YB0G@G$cYgczY!q8iCSI zvq>w3MgGsBi1wgaJ}PX~8BZDX;fp+@ly{lv34+wT-i6;Z>==hRK9B%t) zeKNX^wAS|hWb;6`x65#~fd9Wa?1wmbjhAx^!xo#HfpwrHw`J`t-@Bh8FS%veN7&gm zzl>8#-MpV7!Jl;9;*>_p)Y8;|ZXVJC8l86ILTf{6m5^3D!?rc*)lji_>yHg!bR@Z>kp9GZp&#zLvuYTSoOQ+y-a&lvkjP7qEt@=$JiA_&+}dumkJi zhUa@5X>GuQIXlbkp8NRMTVy|A?`6*~u~5-xRCT{j|Tx~{{S03`}CvS-bicc zG+=%6YHaroqMzRlX?-QX+w+D0A=XdM-uVADu@Yq*SGs2Ivxo-ySm!OPG~K{;Ab)F3 zl$_~=ecG4kKE@4b4_pWRN~wd`nxZhn@9`3#Cepj#x=WSoX*!5WK`WUZ{``};h-&G& zY)~ZZTJB|>-r%Pp>-26R7bmz$%cn#_T6b|mf7Hs*0ksu{g}o*0Fd)*0Cf>>JeGXewU_fSjL$lmMvo$*#GZ^}I8v$T(K4Ap_ zzH5G#cpy=-RCa~bcSR#z*3S8h{Mjvz9~fve+|&x2{9tE!V+~nejt*j+nujXlx4G6( z`*7o49r10o@-Z(=C(Q2U$4m|RpLd@C9ft^n$V zc?D0+2DFlwayrs$fwQ}IZVw?D2iJX21zl43j@lYn2h8HbO=OS|9$jbHnjzVN=!R|P zS(rV>8s;=If;x$#)sk=v)eZ!*b~7)-YzkwNy@?6xbVpaKwgNvlAYqM}7G`*iVa_Ha z3i!EB75=A(G@3kK33ipvPYfoe`WxSMb&f6{^K)t33C?eOj>zCcr0gJep>XY13}4$q z)7mTlVJx?qtC&m~P%rz55A=LbSw05d-E#C z@y+>na#D8ykP|Rvfw|Mt5!V%{1NSv}AQAg*?*c@-(*+%+UY#Q#?LE!3^7-BN8umBY z_tYT;(g4&A;1b)~Gcfm`NA1GJ|Iq5^*7%Pw{6`r69!vg@F#JCv454|U_PGn+%{Ozh6o zr0Hz`JK}zP2dm(pBYZ|EX{QX2e;w-khX|h=tFN>hw?Y8cpThYh!F8<{s>cS<%g)cvpPUAabeu+f>uh*&gmaqsMZm>1fn(!N;Q(c(% z9Z!?+r{_JZtBd-tj$66AZE`3hwlR@-LMFFBy4U^d?FaARR?67=vtS5_PKnvD0;inY z*FQKZ&i&8E_4DsX6O!dYH$ZAm`}AX;yl`~#{MqE`AA*#-%BcF{$0iXHk;Yy?~7Jgwa$63Xki@T@~J zt^$CplKt^t2BM=82!*_9w`|PzBa%-&>9k#-jtIfQT&l;7Wr?JwhdKban_j38YgPD^ zApFLC^eVH|x7zdCCO6lUUQ}3J3S#b05Pt44dNt7XrF-mkJFO`*sg7qXVj}nSla9gd ztgnw8#yyj;>$EbL>fq&?#9Y!EJ++Y({mDv2;cVoEX}6=YuVMXQCzG@BSF~fETT;=t zJbTP+r@lY*pjI1yi`U&M{h{1(^*T`O@&ZhckH<>+VoQ=$J-NnO>dw!JBi22ksrs-( zzqcy9OAveIepW2dF~a@$RXZIUbNgPA?j_Pby-r)6-;EJ|;u;|4hk1ghRx;~zu>!M9d~@}%%K37VqKFs}w1 z=Vf8FLZ}@l?Qi}WND;Ta!LA>8;@mLMtfnO@?dHF8EVj+X&^M~)(iQd$JFDDQw(`0f z^$Bu$4|R3p+ipG4`hh3(y# z>T}@sLs`v-4{qN*SYfPul3g_?^5WHq4>pvh5Tm;vrR!H&-yb{$PdO5Z<%bP>rY($Z z(fs2un48j!w9z_Dpu7D5&|Wd;*%XR<9{aaT8IlShtj5W|Y;)mQ3TWY+4HhP)U z!m&=m95mkZpe>5HKQUuC7&TmK0isj6u2#DY#k+)oE~KO|f>1;Ou&%SfSOfLVHN*bt zHa^6g{LB=7VECmthDsjQmCYJf)zgdUh|7Y;9pZxvAh+Ete=vbM^B+EM^=u>APURj% zwqq!8>oYiU=f=S@3k>>hoqO_0Y4d4+US@Hfvp9)mRwq=6D(|yVhm)C~w8p)3KzF z`fxPy?M&&^q+)Ur@`U%}C1m7X7y{@B-(MV*=jZmE8e0G55#hPdqsv(2*Y~IiKezBP zJ*YQNOqYI$cA@C{h0>iPnDIco%H)lSk{Iw#80Oq9w7~x(C!C+(#F7re2BXhBT?oP>o7WC;Z2zg_2W5LrM+`s>bKy>j+v#Wsz3F zDx+-<^1`{ANm6Md7WDG!NPl_Kp!jqv7db&?baa`)hv@k7{7^8!S=nfe+gYH@cvA=$ zypxz7Y=5ZIwQB@3T`MJ^xdVTGmjLcW346{9=6Aw_r8{y8j}tW7qM5Njh$PQp9$S9l z(hq|Me6^*8qkY1U89m@-9JILW0~k~DsMuz*PXpgF|Kag)*|As^654y}?U~ZY&8;k% zb^%u;B@J7n0+iRtW0#mYf<^PwgHq+ab`0t~9YOO?o=RObz*&**Umv!X9>nc(ZDWqf zHsiQU1k7hd0|b_u@A3E)h;*ZF{BV(3iCWXE(_^pOs3Bh0S>Z&2KPoCqa}KZ;Mb``$o@@v2XT+(Iwe4Axbm|5u| zlyI!HlNYI9g_Zl(d1H|Dclj(as-Vo$ANqrhl$Kh;K?^B76D}$$LrpdLeT{Ia`8%VQdj9yav8o0RMtcc~sWyR)E+Q>w#gGR$wQ|%@so>AC>rd^eH9K zO{jXBf~gntVRD$Nj`6HZlXU>2Si_y~%UwU!?lAEn_$LdwP6E6g0Dr3^$!VRWUQ{XL zwF>zWuruM$o zLy#Zn@wr6^!T|G14aycaEJZSvm>a%GXJURhR-NoCXx`oD?^y?!FwmI82GIQ(j3B-X zSKLl(E?hm=8rpFOW?Qk@^wGarX)`u3D!}FR=SH&s#z5SQ+aL|vnJw!HUuRD2ePA$8 zO_ifZNsh5L$5?Gl)|0FflS*0Lb99bWyUrv@L{ORxxD1;U5HtTdJ+pD3SEb%kF(>Ot zgp_0NgH&k+HJ01;qprs9@2HN>OFka|G`*ww>J)!>{iAgP6NqGxttMReKT-jFCUAISUecM< znCJA+z+^MQ06Zj~EbEF><1A_3*|)*u>|pbErCOZ4t5heed3^efX*&H%GjpTa<+I2` z0e-1Ga*&XP_^SXnyl)xUqOHX)<^RDjxn%%s)$p-iwQmrv1uw3Lg?e^uC#u!9Hnzjv zb8K2VX4;^y`dQ^n!d0lbEHNc4tV00JtzcCr0fG}?(Ql<}KbQ~V&9xt23GRMfAw*bl zdq+9z_*}RmUA=*FJ$r%+U_a_lT6073E7bhb1Fn;^SlxvG-df6-7CP-KdL-3G=%Vek zY!6?9#h{s0XnC{hw4_=6p?j zmy%l{5RhT$pl4hu`~c8|$fXP3GDw`iuv(&l+qo$y2%jVzoRFjU9O^pp2#Hv%TAdq0 zP9|uw3#EVr@=j|fTAXd@4i+}8aTS)v%6C<2%Xi&%UI<=LDL+>>Zxa?o-#BDn;mf1A zvl%({bofRe@%Zs-vm!6S_pE_JXyl>gYMj=o53F()P+{C%@N?=C#~&USVa;#oP`iAaJwmJSeEK)bL zz=D?1za}vkxTNe)8ylY9?BUQjNzK+&=wsz#T0uJDE|n+v#z@^Ne^q^XknRG`DgYGd z_OpWKS6E;>6}t-!l{hi?fd+|%kK5yFaN@m8To?a#o5)GDRq)_JyYvFmliTb4Oow7! zrre&dWG<;E^)m5Q^g`aCtZ@cGS^iv%vZdfNx}+PS4{@KoTPk_S=-)rt?G$}B?dis; zbQ_qy(6k@0ZZ4N#%rni$xmYa8`Tl{p1gIyHRw+z$Zpj2p7nrdF$ar-!_DF0#{n>lLwi8XDFw~nhEVSz4W zd~L;?F07b-7bq*OuaE*VU2LrWP7otnC0!0YxDZ(wP?X=bl`?3}C%B{gY=e#ya&Wph zXq`_ipHADQNl1P)TN6*rAS|=~5WS9sj8|3KI?orpnQ5-Lc&&~R6PP>9&+%voC4VJ= zD?Lk=yQt#0UztEvbDeTyW*9N8fCT8SyB}a*>NVAzOWXr{C(?C1Jt*5Vx;sw|kJ}|? zWK0_lsGe%#@j(wCk{f12|1dvqsjE_oK#9Bp-~!&ao>d?~jyuAA#lH0My0>&EO({r- z0^i9iObSIC6_3xiDPxv0Ok)f}f|^PpNP#mGxHP(n5L&l}fPBN*!`ZZDTJ^ zN^R3jEZWHNcE*V3;P{_EZmLHX@G4G>(Z=JdjYk?)a~NDYSp|wUg=C+6@si_+W{s;WeKyp|!{3 z_E(fzjp9^>nmnn=%1FhX1b{(61AMCe;mHW+SqB9kj4Xk-PRO|Y^!&%;i8z5TL0e=m zzxTYR;+e!Jw+C@|-JW|(UOSAx>lVI(59~4F=}J+gk7K4I%M}yzXA}!VWI&qEs1onm zgWvt{O-VgsKK;j^xnvs7R>vdqlAiV*{1IrzCx&9+3e=YSE^V937QhzJbt+@akBcoA z#Q8nt%um1-N9rB&_%y^=H{hPQh`SDeXYxX}hD0SM@D zNs`S`4jy{~LVC>yawlR}mc{|T3jfzVKetbe%SiQ?I;3v{;T4t%P>CPKSseW3tD6s~ zH4AR+8(Hh$PVSuP=IUngE=?Y|T^Sh&D%zJEDz`q*h#`+>Tu@;GxsZ{5)YnA29r)H2 zVTwdtT*TW63ryC)!~TwZ4NSale7BiYD(K?Q$3-(iqsH1<5UdJ-t%JoFWuubsb+Orr zS&u@l9|moys=+KXk^P!=BE8t6yhsiuM#yn#UYhnuiqPGym6~oOOt(^&cw&2&P zsYPL-imLGKH7!Gd;{lHviHjVd5a*;=&XD3lA`ja4W;5B|o~%!s$z=`f-R$~flIS3f z{NPvKeDEQZ4M5TCh3BIH!uP~d)Uz7hLq@X0pNd-4mw1r}Ro99$*LSvHrjE}Qx~<^z zPYPPqErluEq$ieaE*b@JKu2-h<$ANruE5gBXgo}O}Ny@wwO8kaS0yos>Xkt<<8 z+A2**+}z1$S)O6wvN{Oygw za*sb3!6NXn#!r^TCb)$5@W>imMxLmSj&;6fSZtKcVHtoD5N(6Wn2b5h=s9^B^eEhI zGUnB57V_7voVlO8>O5qJP8iiwp>k)4)5I##ax7!aQv}eN&T3=aRY;a8FqAprwSc|0 zI7Zmd4u0g1HHzH5T^xN=oYt8g&$k@%>g`+6`?%-x}1JcjW7$aVCKAYO}cT#8m59E38+YF;gCCZfv4zRVa2$4YuPH6C5m z%@S*O)#X{U?i}a=_?Gr;J?EsTckKcb=E~d+p$3#Lk|6w4E~!t zqr*8yrRe1aNTqiCYe(=${auvsoJ>VBR(MGtIigajHAjY|PogKjx?Vc5e4ea{)Y5y_RFBpx5ciR~=x<-;{bb zwI6xy4EsA}XJ-#KM`kL2F@VYgx*RBiz5uTCU>_`h4RggzHl?HtBTiklEJ<)3trtYr z`zhn{CSZdTS z4oy4(Z;npuJeL|V7KFu8(7l;2(wMwQ#;a{ir{%ML_hj@lG3e84OBQJ~ldG-%v}KDs zJZsd&kTVgYq{WkB^OTd!`7>W7*OFtgPQ9#r4?Dn+$J(=yiM-of-1Vj7H({dSkL4gY zK3w&C1!ifM`HxWJh8js&`IwKaAv|``y*L?NH@#M^d%;2b;t(kK+r;`G02&XNCIsvNbfb#NS(gm3NYuS&O%^h~EmgJ>9qLc;N0KwfbpcN} zivDErz`vWUhrRYX>+@BWx%z5@c8-5H_Uf)SkYpXas&V+=KNT1%Sjw*g|97*l&N~C6 zuCmoM?)>A>x?iUU{xAZD3c3lOU4HTF@BcX5>ml3u8|Nf7Pu=v!xoqZ-H+ml@;pXKvQm%&xxm0MxuduK+=^dV( zCXxKft}~^1B{Oyy@7?TC?dnmIaNPvWVzM;#eoWBem7?^WjEjXrLq$f%iIR7^wL)qJ z{OA4L2(RN`R*fCmIC#{ld8XU6V+pJiK`&O%E*r29x4iD{Jd}7&(m4#i-|XU3WrG7B z`gC5fnx8WojVz@XnCg57{mS`Y$OR63mjkAuJJ@|s z9A@geSI!$eT~Vl;25!XgP^N!pd$OSkUi=))X7|HRCFfyt>+-@txn@B1x!)l?^A$h8 z`Ox*FXCbubo}JOYDbZxOOS*&N;LW4y0So2YnwlrKINB2u5*8{&^S-dFfK{I5pEfhQ zo1Jl#s&F%ohO6vxEJtss+b|FD*G1>Jst9W(b6*@){*@%kf0xP^7d|>uHAibbf~_09 z?l4A9lw8G@XAfs_;%hgk%yp70j=^#Wb&(c-jAc#96TC_UeyR>T?3+fkU4av z?PvV>N9&9d)253^DWA?2*V)N~)cHc*)uQ2rt>ZCoipzPfHu#X@s$Gmb$qHwIZ#>?{ zyZD=}W3Hm8aWMkRW|wg3CV2UJAN0-r=jF9VmLW?#$g;NqS6vZeSGIv4aXeezq1z3Dq1kGeL8qj@^{-*?$q%)SJA5@2*@GK$=C^Bv{9)A%Om^z!q@ z&3Wdylse?p>|l+Q%#c<$;$=NO z_qk&_*-}tdKe8PFN7vf;eB+|vG}KckjzP@^*<2Q1GP*DRm-Xy;7*bx&ENS!8_hTe%WgCByc}Ke(1umFX zM4~37nwqPb2gWCef^KJ~NMo);nh~rdx%N?qOrg=y@TnTJ*6-QzZJj7^S_^KYc!g*o z;7P!dxrfRKzVLe?J6qBknmRpAX73gFya~4iO(hKH2MZlyXd4_^C!uy385wrVwGK!( zgnxC?Ar~Cre7WLZzo{?_+4g^9P-LLW3-v6*NiaLV=jB}tn~)(mx3Ln6177nB8HpqZ zGuGO_JU6+GaV3s;hv>@GWz1WTtJY9PE{G!5p6N35%vdP0YmELCSge*kI zQd`z_alWjJ(5;nlFvfa#+fg(a&KCr0xY)jVwD7w1ZhZJZ(6Ed3R z|9K&ck!7e>PlFC!Gvqg+db*=?Qg%VgGTlrx?dCA!*aONNplynPka6j29t^BAEZSu{ zgErokVIHbwg-sndc(`$ZDWmA^4KRj!Voo6=C?{H#?0?Nd%I`EZf9|Hyei7IT_xNSA! zL#>$pVpc=Zwc1!gx?HL=`jOR_*pabJ&HyoC^IhjY-L&gxi~F#Tx!71VdE(k^ z_Wsz|0$vASoS0FJKCOsBq~NBf*mU3c%9hodY4Ut-@^@9Gy7 z(F}%|xGO-kmKtibFi~P)T@811U+beY zFU0CB3CmHYL($T~J3U;6AFZ?Kx5LZSpEFtV%lVQI3*A85|LD{!0(zXaB?L~BS)lE) zkcDt{9V2SZ_lXrrhwr@?=fg!L4Y-2SkBTI+jMxYA1)2?&5kJ%QsJ4-^-dy(@^pUT% zn7pW)v(wL38VW`0lIc}EIyIxTOlf`R+#zK5$;FNN($1+eUm$9E!ck@HjB{3Wn$%lm zgazhg!Iu9IxO#W&&DG=P=Ok`v@)|q&P{u|N2=3oiC5*HAS;7sz6g{O9|8W80xSf|X z#^>6|uxaA|VehNMqU^T5l@N&B*r5Q>rly0O3ly2!}P(+%c zyQF7e=wV=}@5bjj=RM$gU(f&FxvsgmxF_zt_F8MN{adlu-Kvb9^|y1Zx*W4a!@A^s z!N?~V61u{epy`>8yF=bj{Ln{DjPo-8{n3re+;p!v8_`0lmDU}D<@vg|eXs2@aHs^= z+`f+5c*7Sy5O;HKtRnKiIwe3~j0$EPmf)n?jX?{La_^QJtWvO6 z6`cJK=qXV%o+a&%``(R6EQez*z>X8>sR-r|XyKk!m|BB`3|+-Bcs=C!MyFm5HO50M zL3)##B1^w^%Vy{T!j9V8?A_v4oX;_ln~nR=TP%h}T2E?FW5v2h-rgDw-YZ_b1H>ZE z`&fh}O?-n@bem?!R$DY{HVxq_#xunCXr3tpiiD?Qovf(PxCDnf9oQgSOPWfhQkeZ0 z1un{DH=NH!*Uy@pTh}hG-RAHoVjWM(>XKpJ*LQykCXZrc!7W^(8!`}+nsi^*Bxe^M zywlVc&88^6nW0x0nJ{2fWYQ$%d&xwp(czZK&acSxC#>2Nuqy&#G*O{p!d0;RQ{r*s8jU$rs3 zbuun%`nI|mK5~Vz#A?mXOWk(`jqu}e(}yy(#mKXcBvE}voK*eo+b?@SYK?J*9%$(* z`#Iw{9@JCy9muZIVc>C8)MrPuOKcpE_lvJiKk2Q@2?dyQ+%B8b)CifIKOZUfO+Rj+ z-kf>mVX~-RWFkFKf>fXP`VdrWoprg3OQ6oe_t5Tx$OEThIeD?8QnOF`**NP2L@TjA zv97#dQ7GZ^cKn>ow1V6+n}P6~hVhw+7h1z?FFet?xW^4JUio2^xAU~*P#|DW+E8v& z#*n}Z!oE(HLfwKnp}xmQhm7wFLMLhnWLKS6p#HbX1`uzG;S;9z54iTaJPX@v7Z=|f z=#0&`+gHJM@|`j{+wQ7Ewh_;e;`Bs6FG#x_y+WmKZ3H0G3o>hDrpzG2aZFW5d%}!s zNUuvT_At&+-ueYS_*Nu?TG1tht3y-n2BE`Guzzi%KC2Dea#$&Qvu3@(=3)h2$w zyQKq6(vAf9NHBo2nnkOXBy8UlIcy9<+mbo)B^g&9(i6PTpWp zh~Zh@h8-WZSKW-S_e~$LZRZ3{IH`fXAx1Vw+qJ7@FCaC%=sZiCp4#ZRWYImAEi3e9 zcU?+vFKf0@vhT{sPJ_g;BoX8Rb4uLo0EssQyWUvQnMzddvX<0$#W5G#3mFIewDzs3 za1HV&gUxxKjkcmYir?{YNB z)e%5)gt!|~LQEW5;K?4#8pjKHW9VIFl|iX(A0$g? zVXd}2WMs;cmuZR)w@g9^j3p4%(m{>KZB_NO**+B=-9wHM4J*<3VpSWQyoL%A;TG6~ zfr^xe282e14z?*wzRBQiN8d$L_Ipq3h4m6q_9qn`3T&(G_dDvr+GxS58pKMvkV>UW*pZ$V-A7@>ir_(Bf z5N3pF-l%XNpF^`wZ@ar-FwK9st*a}h2Q&Ve8feJg+5HA$RZv>W2QPkZx&5W`)Al*E zvi2J3GwggUgGFR~+43!c!?Y@%!~JA2^QCFq4w68(EJLr+?YAC+MxP1fc~v0Q+6XQf z`-8e0`Fszu8k~KYKxA{ROiA&DW`Ez$=NdqLYk{&gcukJr@`{&k?qnO5I+0hSu$x%a zOW{Y>e#bz80o>CKURy7gO7vVLtv;n9!@$P9y`v+MRuX{3AR`ssWl8V;#+i4$xc$PQ zIxxL@CBQtV(p;oYfq!WiJk7s}Fr6R^W4q-2E$JJ}^7FpQTJw^kUOY}gprap5OWG5gvVzK=6MI+*T&Q=!YlH28e zS7)KMM-$|3TL0(;09y-J=qL8y)=0JB(1E>3isLfKiOSn?ntE2U2~@J6YA~+h9hN!J zH1a-Hnd00LS>nLk%ET&lk&{g0EMaaNGfi_{H4a*h`#1S9S4EeDb}Sw)ahp28w|IV* zzwo$z@u`H568cN!&+T&qI=%Z+JP3@~vaC(`3JpTE$Lc2V$%%xzQYV>)&4x1GVynIh zA3mOl_vD-6)#OsVC0YMgT4SWTIPBMa#Qdx_YL^qbh6@e%E zWWE~S86rW!S3fTDZ{r;5bgfq3Re|*E*3Jq@vP+$fwEV!+zjlkzwXplKl92}rAK|JB zcA@FGW^=2pD#vXWFIzvmn1Y8_2i@BbYUCN<8gQuDv){VTpm2eRY&2b9y)95PwB;FM z_)|&H(GgpA+UnXvv-L8^eFK4Av%@~PXhdZHp(1>Fxan$ACXA#PgQK_EeviL&WNKmZ zJr*hrxs|7ujx*)f!}JtQzi9ZW$&srIvjrsI;-DOonrow;t2OGJt03_%yD2$B%V7m% zzcsB+j%DuAW|rqrpfskEBZl%Jv0rZ=CZrE@0qE}=JJkz7@4xcx7b@NhMzuVM0yGXkYeKYfTyQ$@;hYxCx~t?WS?KVE+W*v zxBqHgMl4DNRIsffNr*Za67y~0f^l+Vw4M#GGkYuod1WAL8}C`&6(`)ceZR`F`~K|2 z`~kHE?D^6BI2&Ir zWfMy^H21|stL(oU#|hfzDEgRX*jVCDjg7WG5RUSEE7Sh10ClndXlZ}NFn#opJ7+=+ zrmoGBvpR6tq@9e{p_Z}(nhv>}cc49Sd*?f;vX7rvnLbTQXKHpRj1 zMaX4cpREa?_*pa-qK2GuOF;=12nfw?@8nP&>K9Nz>L?~=^W6-i>yQLYpY$c&4d>Q_ zCR`-b%;SW;p>$^g*CcS9uf7q~jk1OQ1P2F!U=7;5Knl^KoNA+mHy>s@u5q}}dt~dB zgmlF5Pl+In>V^uZn5TP2(=%c36SBsc01lU{3?P`Gn)dNy$9{v^xojP-94Ozs z(tWsmD}AG#y-1|VmXmr zP{sK5k9kK9(bck>TTp}AFp3fZ_f&@h36#Q7BZ`ZXI&YLdAqlS zdb)TBJhVKA+QfEX|G?d2b!4c6WJ|pt)QV2?=jJM$OfZCqT4v1vFalHLv;I?<@ylm1 z?KJJ&7w!YsTIrky4FwfUTDQ&7arss~CJQBB@snzDHZdp&U!NFwj+-@3Cd=iWn~mQb1~`bjqB5b0TuuG!llE~dUI6sbu84h@ zqpBW<*=9pJMY?n$cHl>MhV}=t4As4ld^_(H9v#4}w|>0bfh5=R^I3lQrQ+E}fxiI{ zscXz?ck^u~ZS4zF3nL7b&=)(Z;Gc$OKE@A7YH8-HX%6PYtB%v$7TRMAt#B+S+h27O z$b)@Zs-DhXK&MYgjF~xyhUFk9y^l84b?~*(HyP{`ox7iP{@izlEOO;PV(2lle=9ta zp4mQ^Qtg{~piA(oCMZ5n>VjONPn7rjWv1i!eJ&$F~-9B4U;bh zSM2JNDny3@$hqPcOZ-hiW0F%DE!*E-SpE5k_05| z7a|;iX;Eo!T21VBF`6=YGzA~Fi%fFpi6Ue9BqQI0dDI2k6K{VMk`D8~?5uQoqU=z~Bz~&P z#vY1_=MU3+NJBe$zlN=_dvguR=VT6F;aNn)eYo^Zm2O%#mal>k5y47Zu=5bFq@@YKDp4rdL4%{a+ z3<1n{@bEpp^KAE#6I-ICppnaGaPE1$gG(g0(3#(O@M`gZVouJRhTR4e>R+O2zZgT2 znkA#Q93>ZNQ}E*jY|~da235;*_Z8|Zr$@4OMrCgK9b8_59y8tXEw0wyH0i;FFfzv* zsa{ijeY=4xS;pDB6vWts z{Tk_e){FctdrJEe$V3`H$~!BKGr~FN_wgo>1l^KuIZu0LpE-Le4m{B#_3N7_p?^xN za&w6r^DP91;hs&F{Wi8SkkWm9sfhYFmYkW6i6M|p?znlE;4DqQk@6Mq)dd-1sFtq)SqJ zh5A@zni<8ZttB7J$srwp7o2)QAq4#=k-ijvf>WKDMHGM>HcwOUZpc!80u%Q!r0?x0 z`s4VtB!uUyU&=^^^PS<=Zpv;%%Upc5N38}3 zNAq;44>M|M7hm+N)DE9cojlg?lk_h(2LvBcWNP^sPv7C)_wO>NW+DFV${Wu*1Tnjn zv4&CF4Gvum9v&WgVc~WoO7^p~He94L!0meCNn^qd)+KrxS8oqSIj7{$(ZKeWbYrZmO$r@tNPAruo-8 zeoqbvvPeyBj`Z{}zkl@}pc|CjRKoZEuv*3QP6QbpaWDDI?0>r?DFTeyd&KayKMidn z4H$~ND6+k?-s4#+{odnW43bf74*6}YGencH;jk1%sBV5Z;U5VMhan&~OsH(RH6%&c zYy!(KLA*WLGn;M^tn1jvG1R({pYW`LOQ*QBzd&cq^>DJzGffxbQiW`@oo`O|3Cr%I zDVi9}CePO`5C1td&MbCZ44_t@?nUzeGP#H!!STE7`wl4Ii14C5OC2Ames@)_b4^yP z*Lsg9P)-hBWMbJlU6b*dwV~|uegU~;RhMBEM7ke&eke~hmHzI@V)JVRR(!TT%ZS2E~w3M=sw`f@{Nk(&^s|$r1$TS?$vTSS|77L>l>?aDC|fOkY&{_ zU=5~xP(IPsO0Jo!0Xo3epI14IhfFliW-fT_0YT?>btjrwfhiw;c=hv}{n_g9TA#)X z!QUpLQP)nHep&oVt^T2$)oQJ)^FlS2$zA2>M@F8>-deJI=%J=fz}+0Px6Gf8ABdR# zC1F}!fbvuZIiX*)7m)7G4>4?_1&rdo=JCWrBM$d@=bNRc;>>Rg*gZFZ9-1$8f0Eqn ztT2*ubS%D;4hhrB3S$InAY_0>w~Vl(^6yieXBX<1YtyLzo?5xN#~{dW8gw`2 zamA}?$V><-+YIE<1@?cb-hw8ZzS}`fmW7rIqp#`|&oK%9d=oiPsOuYJ9SXXq1e63( zPjKBhTpj@0!e>2oXKJD3@5qO?MRPC>>4Vge19r>_8tdpf;>j2|>|i*r@3`7vSZ}mK z3_KUsk~6?2(vGs=;U(Uk^B6SUO4BWKMtDuWWxTIjKvk9zrp==3iX2LgE z8~H$SY7c9e$I)lIz)33r{AEprxGd+6%|!7MsOCL+L3-yTz#~4??HBs7>@N%RhAWKh zh;!wyjrC;-4xRs$&8wtj-;E;YHE*cbaihS(o=ch)_1bH8{?psU&>N+Yyyv@bH zAXn|Y%4oo~Vvh4;@K7Z=e*eu?*un);GJ_iX;BBhzfh~woWkN&*rL%X+kU7<<;}YMIt;t@oAb_#?YaBO>Lju4ZW(L3r{zj zjbQC*vK9fd2Q zLpq-gSN4dosm?s6?Xz{+EwOPotjT^HPEX1=uxvY2Fgvo+;HNl}3Q5w+j^GSc@@(5? z272KKE{QFFyU)ThvXT((qJr;*@z^1JkXqsU;_H@0NoRsU%jc#m19b{cAysA{A}scR z4pAblD`PdX^4XtUT#N;VA#ml6)3}P0bXu4DxnCtpmpfv4@0U(P|JG?KM)A;|mhs^$ zC)+YD4UPlQHE0PYZyl|~$}!+Af{qF=YTqUqz$_>NC(tsK;uGcpKPlo;3{VdL6smWe zXZSHQ3Gz;-)D@*`8*Lg|9lHo5Xce)Y%_Aop7;~5?A6Ri6 zwYKzd1uL{8da(JF8UJpQZ`m8lOr9CBA+IGrs)G)Ry&qIhv0Zvp5K=4hOGOhecboDl zUx{fvDHR=O|Kh#x3bZe%RZJ35c<}5A7kXKMZ{%>Z9o#wljRqDOsVpM>BiLJN0=7kL z-Med5a**Hv+Z4>JS*ry*pDXq#xnWUNf74n*5ev<*1KTrIj6263Y_?x_VJAEug9T+d`E!LYMD-e5WHZ!0;Q@$ia&?A&@ii^Wt z`6LP^RJz8FMm~tr<+qKa-K<70D~664FVlG&?3@OwpE-c(t{AahsDT~>NfpI1!5mf7 zD*NGJiNa5*2EY?O{=!+1AT@$t3-+?UF>va%|B(AQ{JPDp;R}S6aYM&5?cllSA*1_S zvrOZ*uimk+u=wNpOG^>Z(IoLq=84e@N=~e*VtSte6@GoVPIZ^@>H&8eRWuwNvh2{U z&*r`My%s}L2npcS*-Tyf^g<8VKV6Xhkw7JW+riC+Ho#Cjyebn~J1NGaxc-C~SHILM zXX)u%YWL?sl!k64OK71r3JNW*C%%FkU4F-VV9|YXLr+sbosr!v#vd94(*B`$Ok(i$ z9MW^PTHc&OzhoKbh60LQ@|V_lc!tpjZV)p%mv##8V6VZrMPLIfN74=D#o&ehGTeLQ zzd}(B4v%uvwAC@LV=+@hPycqmH!m*jy9{P)YHJl3epaLY`pCKVBykVCU%dES-apB&`m(pm=$mb=AJSD?}vQ>_8%DxrND5R@fUQs z#&_ePt$~~j1)%T@+5AmdY4dBrC%RVL{a?%Zd!hK35(cG^t*%X{=OfOY@Q>JW0{|Ay;+ zZ^&rx$A>j_Nt_<;UvBl+7-3HUJV=^bE2w*F13ruC&*NQJ0}$Jz8^Q#^r{w(`rQ#+) zz(*->bHVB~Q2p)pj~@dF=913J9lpN=@f&IXu=^hv0LQtgEg|(ZK0n3cUsm!i;CC>Z z6*>KviT_~T|M2v`O#Dq>{;A{te`+EvL%eK~Vg9@AWnqX@kCOfM-}5>- zs#88H9!X1~ct0da8TwEi>b^oD2$YcbevzO{_5y8Bn`LWP_#IR(3A1RI=ZynxG5}BF zv5iem5C!^W6pOm<4Nj~fa>$mMctUKz(g=FS-q4qy8%p?KaCGN%5Ql`rFAIIkpn97o zZmXX(2eWuFxfXHGY9MD|9xY%P!(;)D{1F@ybBS0fvIM?D3+(J zKPwhC3Vn~lRwVt?T=T2d>SPGX&vibKP_1*^_<3kMlxJE#|64>--{Eh3@IOO;zDrU@ zo$4CLqE}t3Q)b;R183ABAeILrV)W@&#dlP;21|WrFYun6skXiL*ADwHqS+;%hV*e= zxpKxTKf}929Hi?9?@e!WFAj@}q9-7_6LwEpDq^6PpIm{%#)21PsT=5iHs;e%2=LuU z=gIrin}IRiG|;quFCcv(J=D&-_5h7pQ{&Drt}0y0=)5g8<*RY5nWv&!{%q_X%ZuJ*FilMyk7#!nhF=*Y zhcCsPgZ7~Nv!7BLsEcd_6^L)Er-sN3k%2{UFMY?NGiLpICTiST>%BdJMj}ab2@ds9 z61astP$MsNTpB&oNy6Uj`?{X_U}RsKpg!Ws!cgN@*B)P-kVj0Z{eaA*_p#DQsdZhR9_&X)g^fp^Vgk?DGhXi> z^vT{HuTD$r<;xq>kVwdx%_5A(FAH<4&e3X0WN&XJFjV?@AWKyNc+M)*$Or*wOfi95 z)uJQu9s+?Vw&=8hJ6imYhW_>6I&jxMz1JS6cYMF+_@Ak75k{S_X~-ms8qL2FMZJ9X zm)$t7Ls5X{i=Fv^-7kwc-Ik$)?hi=eBJDKLKR?x;<_y5G6N+2Sgr6>Q&>)#Ug zFVlHx3doQumEq|h*%Js%rXmV&?=*GhC(t5-~Py+|0?7e@t(f?uR@;Y#~H{gEWab9VgXt$h4b4U zM*@|TlI0Fl3nd(9AuP!{Qv*6fj6CLajP>F!DRUTgD)98x*po9>Ks*ILHQ1TJ@ORg; zwkQ^kldfgxW&SAt)np`B^4aWIu%-wOs=LvY@MfDdzPm?y7dQs zQuY4P`QY#rra{wRAUt_N}6JspA)dO^hFXXQ_9R7(sNjuhcLX9C6-X^l_BmUOlK3~X_{LiX5r zd$avro12&y9gvuNg1!T?S{2aEm*?M`uSjNKV320j%3{_lw+lDbqjA`*a74F_@A1JE zVma8%_4^Y_9h=&Gwpzcn;NK4(@0xjgN|6lVhf=Fab#qO0~^r0FGgSrVxVtwq^ce8=dQP|gW1_5T+ zGHg<_gg=J-!{s{ z6AWgl&~}E&zgkm2GpIf_3y1^OpW0Rz6ogWp(uM%oR6fDF4vB6$A9p346!nw;A2 zXeID3mo;1c`}j7$MBuB&GiLpL3;-ejQPuEIYf9^bcXAtlr>@w!IkPJC&a!z^j(G&P z`e?>`j7tWDjtihMtm^nARvH^oOX?s$fmG1x{kw-so?3pAUly{lJY z_u=8htI;CGIG%_iw$v0u&d)ME~FiKgZTTADzR zsTnJzB`Yj$Z4;eq8>LD;`N{4x+4$o=U3%#nKyCC$(zRX4hh-%Nwl!0BeFvnm)tH5_ z(n7AL5Uo-|OjLzfrziW{x;I`Ys9I4G?A;&bZ4v{TSFWnr13}AKs?nrik+PR#aF5)m zR|XE$55G)}^1)5RFI#I8&+8z|Qsg20BSoeOLlH}zJ3#mH6L1|%w8EmNLAxCx8XFI@ zuM?89u$VLk+M1ZB2L(bRwF;i-Jg^%f1CbD@^@4am{;RE|J;H;+_UXr7k46{i@whJF z5QgtZ@{e`cewRHxi^n>nGzCZ@NiEO) zhnk^jQz&PP=80(7Ra8bF0)Ce#tA_Hqs< zW7E=S1Siq5X>)|;ZC`dTd~$2_F>!)k=7_aLB?lFKlINN^ejeFqdpNv1@zKO(xJvq? zF-|QJbkBEr!&XZx7q0&N^{bk!rKt6*oh*s@_X%a$cFWx_2wb+jBGTA%*n7abJ%vVa0H#)j}7k@OXm+8|C_>~?^oN4E#|9TcCAB< zesa#oeJn2@4jgFFDX}OkV`il8%gs~}k?p*aN^(EAK1@CU7f`TlcG+9LEwI{VaD_&XpaId*6DSS443*k9(;*m8dmiQhJe8b$AlX_`}9l(5nub_ zLo14=5ggob#MgdaItR6vnF6IEnfPFMP_5GWjLGXI%EZt?ui9XIu)~MF2i}QfEPbG& zYN^qy2m}_Lt0hN5fJ1xk?28jmq;op@`;tR7Z6h(sx$|dUxA8>K0;Y9MS;3xC*UxvkWcGja>|r?cr==epKU!`VciU#>U*cOtUps?-YLPM zTly(ar&vAKey#U40Y-mjUUox3LG=)##B4Wub$hw6#_DH;hchbP6gBvRP){Y;ri{g~ zcGr1&+J}DYNAZWQ{dZ(D+Z@0+nCnDK+d%30yqS$snd#Ux>$$8wRPV9PFO z#wR+gSc#%3F4#9h)o6q4ycfUGd7IHR@*&L4rHa@4kIl9=qXRjMS;P+HMCW}>aMfF~ zL_Vw=?-;P^)UxwC48}(=C@I5^H*Wi?zf-kZnq4Pt+&Pew@n%VO^}t4Y1-zGvw*L95 zP`&Yj5{tsZ&83F5ed~>gDl$h=DY@k6mAAkH8J*#}V=NEWx0xu=4B%^`C74`TTjo*y z7aBnv)o8h@oapgSFbZ#P_icexH^uL_p}Ry?^+p{?5o|&aTk4|;PJkp3McJ5GvCS5N zuAX5_{D4P|$f2C*TUm`U0(^&eNWm}la@2oTQO}VdcFLdg4>2=1V@ge-a=Y@QK)^2@V!6P8( z3zJAitWF4*x=m5Ds^;+p-{w!6Tpm7%Go}2;oLVMRLS=hV!WC7VYiW+=Y^w47*4Y?Y z1BNz=6PacFl9_sKKc;b|X8Tdhv;LZ)bJgw}B2UfLENVR6+MD+eb9NrQ9eMEB&RBsBI8iDDhS7SZgVUuTN7T4^lT zl7oMm?f49ck@`aXk!x4>!|6yg;%pwU<=U-j_V5p2nV|m_NFuReY0i#>O;yRp)kYC?OzN8SmCU z;oE@m(F$B`MXXmA)^)g7wRlHZwO*Vk!KCeYb7{a%s|y>btAq{TnO05r`ophtGvBn1@bC zuk!nXvi?NJ^PBo@(FKoR%>GEPKNL-wWjapPLshE>Bg+CB#_2TGKd`|Zh-sjAUEOVbE2 z4^jcDh4S$|>dI~VAQsLDh^RC|n1u|R@H3QA2)n-IAW8+Su@T$$R2aqKtgCC>BNJhvsYWl?#~j=4~INHy632D0w3fPrvBieq&m^gRIF0*48RSH_MWqr#)?$D zxQ2PJ{A%}9sSkTNqH)BM@X5w^w-!S^>O-YEDe3W0ML_q-cx=Ej zsGZQf(SuthS7Pmc8MFD-)KS-B8@OI86jr|Yx4qshwZZJ?dcA{fm`p*Z^nyku3=5S_ zFDsLD*aj-AxB5=N{E-4V$c=ves;w-(D#yBxfGQ(Ir+|E~?cNo&z(I5?bLf2F(3gAs z6mN@7(q)r5k>hw?1~1yP-vV{s8|QXP3JXKn)q`I@6Lwwt@yx!*P3)PvKU&dtFi(=$ z#XeB-n?8!sgY(iJox0Ihe7@c5Pk!0PTPDc`;) zuU2Tv^cLyH#9jqawgUs$fN!j+1ivv|?&{+oMBl_F6o{D243!l61zwORMDl++J`(z9YpfHEqD@Evg$7=L~}B&b!+JLX37 zg9+_ZS^%0X7SRE0+Bq`&JI9Yqn~77~W4U@8jvD#Ay6jR~Gsi0n<7}SaO{Kgvn_ua_ zzu0ZmU;08pw{~4YK9VuX(V7xj?&F%$YGpRx_dgtd8qei$9ywShfZ8(`u$#n2e=}q7 zq?8@eO~2B==B43o^9)y7hK?AHf!#`A_V+}D4H_w@Wf0Q9} z&%)hCyV312-P2lW-@cE5cY>b*+N@E-W2OPT1#HllOUut;Per0I6A7e7qeV0{qnmjdKAx2#*(P^`BwB;IBP=i{%S#Wl+pOstA!`=}P% zihwCpI@=8!fZdB+Z&YgCH<-z5?!~BNrI>aOl4dH0!MKAjLH+1cRe$Ofd66?Xy!J<@ z(ztoaUb*SpHVBquc3NOZR_-u`&8e>nGJbp{!;jdkMYU32`tBr=TON#92wq;PK9qkG zi38Y!d&xLmL~!v4@KY22?An|D#1sn6p;YMr@@lnaebNezktx?g;A{xfX#N^899(x* z7ex@^rTaoW5rpGv*2~VcEfCx)^)@?(4x2!E$#V+5-v7fB3T&ii9i4~F*o0N7JFrx> zMHJ85Aj%Q}A)8EJvd4DmSuL6CWsST5jUR`fQVyJO`mw)%*5*}uir@gRccc}QSLpXj z8Gnuqztj#Yx7i~ysCM={+T9XhLeR@yFDX+3yfa--$`}_}2cSy^YOD@i43){;q3J&K z#Aq5~MP9v{QHhk{$Wm*{iQ@5)XEn^@GpP2G$QQ)6PfkS9K7C{}{so_$b63b59upZt zDZH>q<-|9%{TUi2~#}SqDXgj=|KMYuuJ|hqbdc45}u)*Qcoq z$(M~3OiNCa!b zXkk`Mmca=DJgjiMLnS31sNMQ7eGfk4CC`cn%|I_DuC!wWsKXMGjR(rDN{3_p5sXJL{+Ai>~!s08pVZ@7LbU z2HhU{49m{EwzdC4Elu&}ALCZ47N5o_Z+>+%OqFZP&FIx5 z!vA-A|IIrj>v$=*2;HxpJNF+^zwV2p5`;bG{XzBWKZzoUH;}$G;hUB|X}tCyP~UKT zNt_#b48P@e@oW_3-;jO|=REjaK+nsY=g*xm*l$>F`Yrni(7C#!sN^{_`NiFj4ecJA zs=aBgUS8y8#Q(VCdg;Y9#qfAX*mg0T^LX3|u3le@?-k)2a)(P5TT0;-QIVlnmbXr<2D9%tvSi9e$pq}Zgm#-nDS7RmyxE$6sY!j^LBmJT zz9Oh;id`B&aOr&3eLaP+BgHGR9jO4~eSE`=JoEaSLkb})cJtQ60gt6obndnUJOt@h z#>$^cE1boHp;9<<2RkW#XL4|d2h@+j`|uG@_ta= zW9U^s?Bn^8!>09Obg$TM@LpQ!hatoG?POlQ~5xUN~KrBE06sLQ`Q5p3BF^e9*E zQEdj1H7)kYtVya~CZ-V){Z__hSS?;BhHR5JwT&B6j#7pIfFN5jyV!YokmHERM>?VW zVg1{P42MFAC^4h+knN@OVkIuyO*!+U*fTL;$|LE4}9CT(*_* z@zW@7Nyh3Ivy7b|`NN6UG5S|5)`&PJwa2*WnEHTI=+-^G9`r7GI+0o$r7g?)nb7W| zsh!?_3Zju)JkPIAR5=Cy2okV2-Ocf8x%q3w(I&p^skp~D+N$N9zuxO%zeX8t1hm># z2XkXFday&zXlBD0q(Fg9T$Lo^TJy4N*|zX#-Pl{3K^Cy7jPj>a>t5-~uGHqxr8**x zdv`F-#&g{-oOh=h`JO*^uvAm+?SK|_@9DvuX69Z{isTg@g|*iUV6q(Os?0W{e9Rsv ze;6yWtKv8eV^pTdm%rSlYD*3jT) zj9q!2;1Vjvj>WB@)_HE}scyZ=T`DZbA^JP3a&tk(RpGOk;84-avl{vlzs9@`ZfY%gAtb|XtRf|B%d)e$@x?!FBQB1Q9XD&t zjbsizFx|W0#UK5-QGNuVvghlVL;7Gf6pUHFH%(6ACfB5^C%HfB=5(Wra)P-B*? zJ`1}YgsqO(DFM-&&fsGi4Ol@cu5nX}LNoRDUI1Z#iVrD-dr@IEHgI!IGudIra<{%s z7W#FhBy;Hj@f}&l$Nt81>mTwS`@SXGH1<%_>LxUK8(e5HB~O8#4^@my-MYgjxa|T8 zpbh#gAj zApia!juSWJ&Ru%STtjE&i zBGn_l%V}KdFq^D%Kinq*Z0~Zf-|KykS8z$FRk!AgNM{=7LR*7L6iaYYjoOB;% zp5&TlDkrNI;Xc{eTe!`zE4&*C5xe>m6L8~4Xxz88n45?pZ`hZ&3tN0<-M0eZQ;NB6 z{;u|lcFiHsN91kVpT&+uvg$3#*)-CxN&d)gPZn``rMv;0js3NqePl95=dZg6kK`1l zGd&%cEMpbJCO`tfZbsO$Ua=tg-NR7aoTMZ8Co!s7>T$ye*L^?S{nUv3_Nq|#dsS_b zTXh~=@2W#~_Tez1>NJ<*YgCvesHNfoROkg!=b_o9Xim)#$71sQ8zEH3@N_CJlp=)# zq$9v8S13EF&@O+p*TU`zO5e_QAVh%e|26jAQB5uD+jd1o6a}S#-$#@zO5T|1BqROrW#M$m6>T%96(0x#$HJM%0Y-_IAr~Z2}Z*UZ9 zf#oLSlTbl$Kdfhl1kS3o1>Dp_!F#Tw(<+}jwaR&T9L zT0Boa`}Xb9@l3K*@I$$vu3e9IuH+@fJ7j-M+~<`(cJL}qmRoIC1=t_j-EdeQd&`#q zw9Q@XjOEPRy;Wby8Xex(Y>C@u=DLRMObZJcC1+7RIp;rI(TT3~1aj=JPwO*ei3QN; zmusnm=s^OFuMRH^WXZ&NETPrGs_7F5p&2w!XBzJmSInpF;9prJU#X%kMM})gkAlVA zIxl)bx$X_`c;ooE!QQQn?rU)cZX<{9lu_&@`bOSUCtdX&h^qi|{&L3??DfzV#ZFaFcF( zVBcp&8>18s=7`|4y{n!l@k$99MYi*&0lyxO`rClcygtHz+l9t-$GNS*EMbT+uF*S; zidO|fjqkW)tJ<)XS=3&stdqtTN7BP>>AB+S6Hf^^;0i-lpBsvVdS9V%?CnaD3Ghkm zwaV04N)jo>l+&jgOjI%AzCYNC1;nD3Lj7cBqMdM+-x1gaUU=cy*1QfsKhO6 zTvXZMIxbp{T6#yQ);GyByJ(L|i<9oHPC(dUD$&O~`LoAn9VGQD*aMGnqFJJz z(ir0&$#C7~k6YeMOdy>T2yJkvCk9e~H z`0b9Eq%X_7bSCSDwLNaSj}D%=JH(Un;73wa_!$MW^I-q z_ohip5#AG1P`xiN`r(Tc^$H)vq_2@Q7k)VkRjis?ti^cO5aHa8s7{VDvm4AQTH?N6 zv3-JR+8Z%O(v|j&2&!y+e!I81ZyCCKkRPmpa2o?sVdU2)0B2g263&a~NDN22L$(QV zaJwG=w83@mgKmkYB!7+9;VUHn1cyF?-eqCuHe8eEj{0QFi|?P^N(HX$FnWX^6;s;G zgD$#?EiVTh4ML^9w)C!F%dJyTG0$dC<~MHtC=tXot=z2qEx7|i&M%;jnhiZp$J_6gM05j8&BKEe#j9=RW-fG!R-IVohX zm0Z5^&PH&jfk%0rh+n58+Gmt5C^gAAl!UV5%8fzFxMce&qTq+wyS zWa#5bQ_CG~zI9_lb;2=N-%G znm}IwCKt@oI#*V*&K8le7)CLOQaI7~CRncXc}D?R!4_R{vcQ&r|pAC`v)fPhnI z8GGp9d^d)}Ag$m5uSqF~ug~>S_gwWXDFU}%2)vF~EWt2HNOTJ|q1mhnLCm?!IUgS7 zSx1!}TqWZfHSBwTs$f1=7)v8qajS?!k(J(JqOst?0D8z+iVf8bFKUb&dF~4QECbn9 zXhOGQ7w}_F7jETyXN2J$A=Reg zeQy~6(L^Up6wKEh^RPb+j4Gw$J~9tL=Ig|Q`rK%8>zJ;uR2Wo!IsMjVGMA;b(J)u& z>1@fL%)=H7edbN4hb{EjlJo!d0`JMoLUWsTV5`OE_R7@0)6ZKJ*xuMRi98KE->Kz& zR3^>#al*zXw{w$hh*Ui(Vki)eZCE$ZQ|V$r08z;-DlnJ_D>M1T@M2%msJ73VjXp}B zP4u*mzV7Xa4P;UY{e$Jc=83v$xHwz?n}^=G+9TO}tVjAl7#`PP47Om~BhD6biX zU3o71t)$uuTQD%t38tJoY=2v9zo{0PefRLI-zs; zEMAM6@y9zR*Jako>8dYZw0sis;@E|pg#&#{yPD;UOcM!^@SilPtXq&%+rD#yt9O#e z7cbOo_Gb#+ANcM|($|TMiFI>nV9ZSB%v*w>9Z3DvUr~kV;*7?76QO>gx6?;SD-W%G zeV$ELn}H-KFDwj_FWT-QCgbx{xVaBsa75VV+}Mao=PTcJIJCI&dnI+McJ4%dLAF9) zN=}QI4aD2jZr6=bIT*BNbucklWB5U?ic`%a?c$JiIWEk}hymons1!bHSe1Y2LlP@m zOB*Pe+u>)1$9xTwGOw*)Z}PO$>i|}AqO~0!gOGjfnmOGL^N1yAtXxXcsC4-2lL+0! z&u(1BKmp)v8rP$(z>2axTeZxRsk`i&x%r~->0u22N^zn!y%~b#wVyWa#}&-JFrm!$ zzOWMk7010m?mp%)B}0lx#1mY*(7w)&+5e#C3;pE^40lX*`XS!uI{pwbeHH1bYK*kp z2mcuRy;vPb?jNe=wJKKlufOBKk$mr~h}$cJP+b?(T5*YIbMfi!MZb0yM*AIZq#$=b zD|)6B?3-_)RN9ir4(E&2q^DTSL zWjC%R#E5}MRw4rb+D>j?d&&K6?W$X9fzEtyj?viUvSc_a)Vt~BJtISb>1uu<1vaU-k3&!< zo}Xh3oSlHG+uUOZa%O=6TtJ0rB)GM4Csju?CAqeMcFB9zWx`5xpw|?thoaF4J0ITNYzH zD%*CYe1(I+^~|;<-ICi$JJ!FsUGnu=Z_9G=ee)03Vf02%2trGeUVK#o}fQs8gA=O zo{jY~N9HY%fwpXMn=`0(MEq0FfYFJLppzVmzrWVw3r&`V!ZiCLOkE zaba=RD^7<4@;TzZ?OPS>wG&AxC_Ff}xH_JcewPgnbkCXgVV2I@ud$w_?fic z{%R3}C@u?m1cvGm0))9o2XWZsg*d*`x}(04cyf(*At>q(N*uX3$+g+Ecr`j~I{sIp zJbS4CbJM4$yg@IA!mRGK=^ccu6Nhni3@T@H(&I~`!00N@XJEWJf4Z?f>i9m-H+vKT zzg;~T*Khb*?I}n1T`HE|U4C7=SUYgA?7nepG1=pq9^A*=cQ(7bQZABZHOwkn;qXrI zu|2V0#j&8=@3AV%%(PCc)sRMH#}rFJ#cfq#u^+VaK@B+Zl>SdrB1Gi1;rjL&TMt54`4Oh0>LP`5@qp9gsHh z%~|Hx5Y-T9TkQq zQ<2??eCx2=Qi6(>7@1Y)rwX;(SB6E*ZXZ7qflcJtZ7*xz7*oBf&tt=C&zFv*sqoko zXi)+%ESo6?ZG@%BUqLC7mAsXYIMb_*K+XZDNtmDB&nVqnV#=J81Eq!^Wt~15?5_%N zFLtjR+#u^*AM3nIQ76x5T;rM}=unS{rWDil3BwWX^bKJ1bsno5(8e5E>6b+wE1*&f z?L2H@#HQmON&M#cLE(Z+{&N>Vrll2(YX5wq{iF)ir&NOnU)spfzmPxp(6}Jq->tH!9A;= z+gh-z-K)HmLUWYBC^dtit$Bj(5+;ieJA4%>Mzo*n6 zGUv^a$!&P2Ntf;ByE}x_4E|5Wyh@HQ z#FvZiPVqNrUCjO3oH+d@74rN&Y?w$QJv?Ud5Xz%#l*Q!@)|WkAduoo@3JK1i7N&A-?GpIm+8r*HS>HAo(n8Ydz>iSY8Z=N?B$=7kE8CQ%iu|< zeYs-EKKq)`54-lDr1k~pY>Uu2qs*kJ*Ublrj zC?0=UtE*JAjP4QI>iVo>5){3!7DiNWtf4U&xF8af+>=gi!vw?%-o+@9Sv_NeKCUU^ zA(1Le00m=GIo~ro^4?TgA-|}?|Gkz!z4R>JytVR53!qG^5|L%vgx??+e#1Lp1HZa< zEz;Yu=VoSI5n6FRj?N%eBzG_W)8KY6eUX*P76~EdPXCysfVIEtH1%nQDLpfq$i#rA zpX|&l?uw(6Nu{j4fgZ%o=|hLc%0|UCRXPEB18D7(TLt!!QB1DxZBTOV6$RxV`ZAP? z;a&yZ3_Gb6>80n4WdY)!DsyJSZc;{8u-8xrbQ}dRO07&C`mCFKjXkEh=hW@A6x95( z-l%Z#NmU2)qpK7gznU45F1W;9L^e0?ixo*HtTTGN`etrMUsak}J;_Kc9unq)3y1u8 zU-0${jm+F4;fUQ$J*CwA#x&fN&mR)ZTBDF6<(H0ewTONpR3ip@|_~p`^yW>pTMr-zNOTPKCevkJ~ zzK-kinOS>~B)@|qg)IE;ZpV}@zT?kngVLigT!j{C5!yz0l7&EL$xiz*jCFjy5po>7 zuuq9DH=KEA-0d)t2#4|0(GuTHy!U}sc-mQaU9USMr!^Tt@u4oGn4N3~rZno*usopdN-|slmuWKAnQ&FPKRZV)M zQ6ti#p96SspnOHcxSHx!$9b8r&;6^2{AT3Q)vB_EPlA5l^lr498yvJh{18+8qj9Gb zigHC9nf?n?zjp5eNv%Qv$;g~NU3JSsCgX!jWCXktf(0T9ao)#^0!uPVF%8o|q+_zd zLcRKM_0r3_S9oVR67#~IHoMs8>GE|NEn(C_bn?!EAscPhN`#^56y=zWqG;cYzPrna zjj?8~Q|gx0jsOB<;MX>>2B#o>Y>@3l@@We$C}PnP$X0s{kCz(=@+hNRN*%{@9Gt$V z@FC_W&BAyCkKb6A(X_Z2nj;swKR5F6>k2F+bFyGtQGTGPraT_Y1VgGd&4YX}HM_Mg zk!kG`a+t3Wfq)CFbPi-6gOh)3 zC(27-$q4I-&mqPCt+)PQ`EiWgs%2_S-v4r*w`Qy=T-&@TzMZe)nmD zME~f;@$(iIw6=%$Gf&s-1sK(MydN#jZCpz6R$vXn6Z!q(+ERoe$7)F;lf;AV?`pnz zaQ~pmqguqJWN+d4A566{7fvA_)~^HdRXKX>>!)9Nt#ibm0Y1@DdH|a~bCQHweHJx~ zE>!Fn`>Lu{r{l6GM!d~BpQV%`L?<=nPV#wNSSV^z0HZl+mmG(LrfMtgB}U*3K_M;NB9f9gnZy`R)W z1o-qB7F&7mJIg`OqR?lWxiluBO{di-#>IYvSt3{Q(;dp)P+8CD%th@m{1RFZ2$1c3 z-o#MH)nTl&qKPtkKfFLaky#eUV2&+(*brO-E9_!jmIg-ZgYQeuuj|_KZ)_yq^IY%J zY#hzZkzE~syX9$0WsqUDUp(+Zq?xKhGUqk!lcP7Tywb2c>Uo6*+`n!&W(n}jpjq!S zW%uFNsK=#JRePL*974~#-y--ohS!mK+A@_TP${1hFgNucZS37fdWSJJW9~z5JD;ct zmX!UM-Rf&Kca=$0^&M$LM9AQ7b1TqN$aute{l{w~inVR|UVl% zZTy@kN8=1`)u`{4OMRsDON%_Cqb8gbblm2)`#v8)xr{CWav_prQuXfKZ)4sl;JJT! zo-BC`0+FsGcfwVwJbn~#F zJ9Eaw=m3a@5{PS~>bt*w;3+Ga~Kxt-+g054o@wl4= zeW%M#eBW_t08(6uKXcL*!4?N`$-VD@fea-oA^TN=|k95 zxv!sKGOUgBE_pHD3d?4WQoTG@HIaearm)z4Qd=rWFvh=Wly)F)L8X3%u&V`)1>{Sz zy3co|B;%IRZ!Fx8I=l``Ho1@?I>+`9##6!*^<2rcC^+=8uW)$?*$oEuZN+F*29l)b}X{`7QtPyY4whsfHRZ42z~RX+{we( z&8aS@ua1Y92U?Ka-rSJXaMJmc1=IsZ6@7J5kXZFrTt;`_{!jd`3o{fP1|Kd2eyvER zO}`VfMB2%c9=7{sqJ$$G6PJY^5j|*n`MVvaGC4WdU>ixGg_oQy%0?$NmOW}G#IGVL z=yNK!!72pkkG45SRuJMc*-DlZ(8nJO*zP-ov-lQrLZnau7W=4v*y#aN=9nY5%QuO6?` zsgJ#wtkt1mZ4BuprV;b`J8*)oXKTjgM#UeD($%o?w__4PmK*s--6CM8$pXa~&hFBi zIp}jpLs*@|csyNH`r#?PD_44@VK0!os+P3`hPthg&Tqb`G%u|0?9Oe~^R%UkUGuqq zC+G;@PF3+lyWpmEdZyJQf0C{RyvDhNDw4dqan{U(C_@WbCD z29j2B-EzqM-o<$$%i^&_$%vXsY2_fSY|1qCFb^49VkmO{$Vz%`2l1rnH5Z^9MIj zOmApH){nNp2Vj{b$!f@k|M>Ytq^k{lT18ih^9s?QB^v51Ev5Te*r&MSEs4t^1q^~C z^BgPqJM00}Yw!d04 z60v-atxSvJr?l^F*zg$ydcp`tpAs67X{sCgXs^`m>)54GY;jMWH~) zq@N?;8+tWEJBuc@H5c&hj1h4i55`q^2gEiH4A?#g*H~RNPL`ZG0NPb4jQbtN30bPz zE?RG^#-zOhkbEQi85erWb-S&MV1Ru~)Zq5y(iWZc6$AOEHQd>>SzsNeeey_=a*{V$ zNIod&4`!CsC9`)TlbOMi7=C+_ka%+Hd%15D5#s31`M8#_d8y`sYL2+&B|_!wqUDO@ zmcz_mdbpj351yUjKEZ-|B{w3cq)FI&NIvC7lwW<0SI5kjLgsXI2 z*e1r&-N}ubbnjIAeCv9Qm&0hXFd{D0%efzWj!M*bd)7zW%pQ{AJ~|&)pUByvEnBmn zk@zYzRo~=VpHV%D3ieQ$S7+p7pE~}Y+W|67-0Yv9*?U(vU%59@9O*qw8sp+0RI>-Q z+Tn`^Xc&dg!-q0SQP*di=I^O?D?}wdXph)5GZ&9}3kS)JrH|U14>pu9D6L4xpkfyX z*JT$2J+pZ)cij!4ZXJ-98rs5g1Sf_$%`87bSO`Q5q2=Q*?={aV?K%JnMg3N9`L-5p z5TOmvu5eK>oGLes0J=Bs&~O5~$$S22J_y`)UCNwHkHA87+)ua~CI!lbNLx55R8G+V&ogQIrmIn^A2D2t=_Xli=B=E`0W>>we+YXo zybUQiU4~9CW89w$&~oU69$os2BPKhqE+82>QT7K=bB+C5Ss@qa`v>GJZ)MIN=NrTl z=WyTS|C(-pC%&!S`7t(EN77cOaN|k{kuSl2u_nQ+)!x%=OS}E%+Z2S$f9^wWOI#r7 zOW31s{2`$FGjozKSpFP!Va*CCpXJ|n=Rfb?#Qx=+P4~9DbWYlJ=Gvdw=bzf9V%+}g;{W(~+mj^1j+4MPQvP=V z{w_-Y{mn;R=5yVvx)%~K|44;>?phZ;NkeWOFH-prtMDV`+EW2y%Etx7|0wPMs=a_R zNr|0d9moCp&!+p|CI9oPH0uRMszS}&H##I@y+lvig!*st@Nb~UOo1=fRQ<27y7@D| z@%yX$|4PW5Z#hvvkst;pL=U(4Zfk#j6O~UtuG=a5w^j^!bbhS}@KA@ClzV}goqO#O z!f2hstga6IacTF_2d58?_6`?L6eRqoR5^EDT! z-mbtm#GG*=#O(SfkM8_;?j!+!%WqGEfuq9udxhsUY5$oR{@f)ine*J;9BNOkPZs3Q zKl>*F`g4~G0Hls6rR+NVhaFdwOL|mmPVc-A`G+@Xv!3g=*CU&M^;b9ipNZFozcu=X zXOUV@|Igq3XS9}5l3WJLFS-A{7ys7!zq{cnjO0#X4+~b{VEXTd|Erh&zr<5Ll9K_g zeyI4vzdZy0vx%9ozkL|6a?JN`|HB)~e~)IaXE}el9Q>~l^PlFvz45z18xQYX|7Ttg zOcoJObMnQ*`xeFT6^7ft{D;LNji=l9?hRGOdkzOi_8VZ%qEkgev D?Z2m9 literal 0 HcmV?d00001 From c4290ad210daa2f2dd837bc7fabe4ecbad7764ef Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 19:33:20 +0800 Subject: [PATCH 09/30] [Server-core][telemetry] update docs --- docs/en/seatunnel-engine/telemetry.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 989e8ac0b80..25c62a093bf 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -125,6 +125,8 @@ scrape_configs: # The job name assigned to scraped metrics by default. - job_name: 'seatunnel' scrape_interval: 5s + # Metrics export path + metrics_path: /hazelcast/rest/instance/metrics # List of labeled statically configured targets for this job. static_configs: # The targets specified by the static config. From 0adaef53880c0f7324b052b00f2acaefb12c6d61 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 20:56:55 +0800 Subject: [PATCH 10/30] [Server-core][Telemetry] revert example module --- docs/en/seatunnel-engine/telemetry.md | 2 +- .../seatunnel/example/engine/SeaTunnelEngineExample.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 25c62a093bf..2a96650f61d 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -142,7 +142,7 @@ For a guide on how to set up Grafana server go to the [Installation](https://gra #### Monitoring Dashboard - Add Prometheus DataSource on Grafana. -- Import `Seatunnel Cluster` monitoring dashboard by [Dashboard JSON](./grafana/grafanaDashboard.json) into Grafana. +- Import `Seatunnel Cluster` monitoring dashboard by [Dashboard JSON](./grafana/dashboard.json) into Grafana. The [effect image](../images/cluster_monitoring.png) of the dashboard diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index b43dc0cf0e3..e0bd04c0e69 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -33,7 +33,7 @@ public class SeaTunnelEngineExample { public static void main(String[] args) throws IOException, URISyntaxException, CommandException { String configurePath = - args.length > 0 ? args[0] : "/examples/stream_fakesource_to_file.conf"; + args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); @@ -41,7 +41,7 @@ public static void main(String[] args) clientCommandArgs.setJobName(Paths.get(configFile).getFileName().toString()); // Change Execution Mode to CLUSTER to use client mode, before do this, you should start // SeaTunnelEngineServerExample - clientCommandArgs.setMasterType(MasterType.CLUSTER); + clientCommandArgs.setMasterType(MasterType.LOCAL); SeaTunnel.run(clientCommandArgs.buildCommand()); } From 443a02b21a106de6ea397aa70f2b1f53c4600f24 Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 19 Jul 2023 09:46:18 +0800 Subject: [PATCH 11/30] [Server-core][telemetry] fix code and docs --- docs/en/seatunnel-engine/telemetry.md | 78 +++++++++---------- .../engine/SeaTunnelEngineExample.java | 3 +- 2 files changed, 40 insertions(+), 41 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 2a96650f61d..fcb16dff140 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -33,8 +33,8 @@ Note: All metrics both have the same labelName `cluster`, that's value is the co | MetricName | Type | Labels | DESCRIPTION | |-------------------------------------------|-------|------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------| -| cluster_info | Gauge | **hazelcastVersion**, hazelcastVersion
,*master**, seatunnel master address | Cluster info | -| cluster_time | Gauge | **hazelcastVersion**, hazelcastVersion
,*version**, seatunnel version | Cluster time | +| cluster_info | Gauge | **hazelcastVersion**, the version of hazelcast. **master**, seatunnel master address. | Cluster info | +| cluster_time | Gauge | **hazelcastVersion**, the version of hazelcast. | Cluster time | | node_count | Gauge | - | Cluster node total count | | node_state | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | Whether is up of seatunnel node | | hazelcast_executor_executedCount | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor executedCount of seatunnel cluster node | @@ -68,43 +68,43 @@ Note: All metrics both have the same labelName `cluster`, that's value is the co ### JVM Metrics -| MetricName | Type | Labels | DESCRIPTION | -|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------| -| jvm_threads_current | Gauge | - | Current thread count of a JVM | -| jvm_threads_daemon | Gauge | - | Daemon thread count of a JVM | -| jvm_threads_peak | Gauge | - | Peak thread count of a JVM | -| jvm_threads_started_total | Counter | - | Started thread count of a JVM | -| jvm_threads_deadlocked | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers | -| jvm_threads_deadlocked_monitor | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors | -| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | Current count of threads by state | -| jvm_classes_currently_loaded | Gauge | - | The number of classes that are currently loaded in the JVM | -| jvm_classes_loaded_total | Counter | - | The total number of classes that have been loaded since the JVM has started execution | -| jvm_classes_unloaded_total | Counter | - | The total number of classes that have been unloaded since the JVM has started execution | -| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | -| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | -| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | -| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | VM version info | -| process_cpu_seconds_total | Counter | - | Total user and system CPU time spent in seconds | -| process_start_time_seconds | Gauge | - | Start time of the process since unix epoch in seconds | -| process_open_fds | Gauge | - | Number of open file descriptors | -| process_max_fds | Gauge | - | Maximum number of open file descriptors | -| jvm_memory_objects_pending_finalization | Gauge | - | The number of objects waiting in the finalizer queue | -| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | Used bytes of a given JVM memory area | -| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | Committed (bytes) of a given JVM memory area | -| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | Max (bytes) of a given JVM memory area | -| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | Initial bytes of a given JVM memory area | -| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Used bytes of a given JVM memory pool | -| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Committed bytes of a given JVM memory pool | -| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Max bytes of a given JVM memory pool | -| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Initial bytes of a given JVM memory pool | -| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | -| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Used bytes after last collection of a given JVM memory pool | -| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Committed after last collection bytes of a given JVM memory pool | -| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Max bytes after last collection of a given JVM memory pool | -| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Initial after last collection bytes of a given JVM memory pool | -| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | Used bytes of a given JVM buffer pool | -| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | Bytes capacity of a given JVM buffer pool | -| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | Used buffers of a given JVM buffer pool | +| MetricName | Type | Labels | DESCRIPTION | +|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------| +| jvm_threads_current | Gauge | - | Current thread count of a JVM | +| jvm_threads_daemon | Gauge | - | Daemon thread count of a JVM | +| jvm_threads_peak | Gauge | - | Peak thread count of a JVM | +| jvm_threads_started_total | Counter | - | Started thread count of a JVM | +| jvm_threads_deadlocked | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers | +| jvm_threads_deadlocked_monitor | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors | +| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | Current count of threads by state | +| jvm_classes_currently_loaded | Gauge | - | The number of classes that are currently loaded in the JVM | +| jvm_classes_loaded_total | Counter | - | The total number of classes that have been loaded since the JVM has started execution | +| jvm_classes_unloaded_total | Counter | - | The total number of classes that have been unloaded since the JVM has started execution | +| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment". **vendor**, for example: "Oracle Corporation". **version** ,for example: "1.8.0_212-b10" | VM version info | +| process_cpu_seconds_total | Counter | - | Total user and system CPU time spent in seconds | +| process_start_time_seconds | Gauge | - | Start time of the process since unix epoch in seconds | +| process_open_fds | Gauge | - | Number of open file descriptors | +| process_max_fds | Gauge | - | Maximum number of open file descriptors | +| jvm_memory_objects_pending_finalization | Gauge | - | The number of objects waiting in the finalizer queue | +| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | Used bytes of a given JVM memory area | +| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | Committed (bytes) of a given JVM memory area | +| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | Max (bytes) of a given JVM memory area | +| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | Initial bytes of a given JVM memory area | +| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Used bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Committed bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Max bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Initial bytes of a given JVM memory pool | +| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Used bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Committed after last collection bytes of a given JVM memory pool | +| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Max bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Initial after last collection bytes of a given JVM memory pool | +| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | Used bytes of a given JVM buffer pool | +| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | Bytes capacity of a given JVM buffer pool | +| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | Used buffers of a given JVM buffer pool | ### Cluster Monitoring By Prometheus & Grafana diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index e0bd04c0e69..2cb74ac48d6 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -32,8 +32,7 @@ public class SeaTunnelEngineExample { public static void main(String[] args) throws IOException, URISyntaxException, CommandException { - String configurePath = - args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; + String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); From 4404a31013c1fc7ec7f454eefb8942739cbc0fe0 Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 19 Jul 2023 10:02:15 +0800 Subject: [PATCH 12/30] [Server-core][telemetry] fix telemetry.md --- docs/en/seatunnel-engine/telemetry.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index fcb16dff140..ad5e25cc7b1 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -21,9 +21,9 @@ seatunnel: ## Metrics -The [metric text of prometheus](./prometheus/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/metrics`. +The [metric text of prometheus](./prometheus/metrics.txt),which get from `http:{instanceHost}:5801/hazelcast/rest/instance/metrics`. -The [metric text of openMetrics](./openmetrics/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/openmetrics`. +The [metric text of openMetrics](./openmetrics/metrics.txt),which get from `http:{instanceHost}:5801/hazelcast/rest/instance/openmetrics`. Available metrics include the following categories. @@ -130,7 +130,7 @@ scrape_configs: # List of labeled statically configured targets for this job. static_configs: # The targets specified by the static config. - - targets: ['localhost:5701'] + - targets: ['localhost:5801'] # Labels assigned to all metrics scraped from the targets. # labels: [:] ``` From 25ed5e62cc747a526156ac31066b186294b5a331 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 25 Jul 2023 18:20:47 +0800 Subject: [PATCH 13/30] [Server-core][telemetry] revert e2e config --- .../src/test/resources/hazelcast.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml index 234493d29a6..c28e9c94d5c 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml @@ -15,7 +15,6 @@ # limitations under the License. # - hazelcast: cluster-name: seatunnel network: From 7deecd19c484d26e0d25d7f45d52c75849b22138 Mon Sep 17 00:00:00 2001 From: kim-up Date: Thu, 27 Jul 2023 12:36:47 +0800 Subject: [PATCH 14/30] [Server-core][telemetry] fix e2e error --- .../seatunnel/engine/e2e/TelemetryApiIT.java | 409 +++++++++--------- .../telemetry/metrics/AbstractCollector.java | 8 +- .../telemetry/metrics/ExportsInstance.java | 54 --- .../metrics/ExportsInstanceInitializer.java | 28 +- .../metrics/exports/ClusterMetricExports.java | 6 +- .../metrics/exports/JobMetricExports.java | 6 +- .../exports/JobThreadPoolStatusExports.java | 6 +- .../metrics/exports/NodeMetricExports.java | 6 +- 8 files changed, 248 insertions(+), 275 deletions(-) delete mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java index 030d471fa5c..09f84aacae3 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -110,41 +110,41 @@ public void testGetMetrics() throws InterruptedException { .body(containsString("jvm_classes")) .body(containsString("jvm_buffer_pool")) .body(containsString("process_start")) - // + // cluster_info .body(containsString("cluster_info{cluster=\"" + testClusterName)) - // + // cluster_time .body(containsString("cluster_time{cluster=\"" + testClusterName)) // Job thread pool metrics .body( - containsString( - "job_thread_pool_activeCount{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_activeCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_completedTask_total{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_completedTask_total\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_corePoolSize{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_corePoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_maximumPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_maximumPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",} 2.147483647E9")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_task_total{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_task_total\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // Job count metrics .body( containsString( @@ -204,325 +204,328 @@ public void testGetMetrics() throws InterruptedException { + "\",type=\"suspended\",} 0.0")) // Node .body( - containsString( - "node_state{cluster=\"" + matchesRegex( + "(?s)^.*node_state\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_executor_executedCount .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_isShutdown + .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) + // hazelcast_executor_isTerminated .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_maxPoolSize .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_poolSize .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_queueRemainingCapacity .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_queueSize .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) + // hazelcast_partition_partitionCount .body( - containsString( - "hazelcast_partition_partitionCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_partitionCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_partition_activePartition .body( - containsString( - "hazelcast_partition_activePartition{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_activePartition\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_partition_isClusterSafe .body( - containsString( - "hazelcast_partition_isClusterSafe{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_isClusterSafe\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_partition_isLocalMemberSafe .body( - containsString( - "hazelcast_partition_isLocalMemberSafe{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_isLocalMemberSafe\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")); + + "\",address=.*$")); } @AfterAll diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java index 99a1f15e9a1..0439fc1d77e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -38,14 +38,14 @@ public abstract class AbstractCollector extends Collector { protected static String CLUSTER = "cluster"; protected static String ADDRESS = "address"; - protected ExportsInstance exportsInstance; + protected Node node; - public AbstractCollector(final ExportsInstance exportsInstance) { - this.exportsInstance = exportsInstance; + public AbstractCollector(final Node node) { + this.node = node; } protected Node getNode() { - return exportsInstance.getNode(); + return node; } protected ILogger getLogger(Class clazz) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java deleted file mode 100644 index 66bcdb8b64b..00000000000 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ /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. - */ - -package org.apache.seatunnel.engine.server.telemetry.metrics; - -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ClusterMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.NodeMetricExports; - -import com.hazelcast.instance.impl.Node; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.hotspot.DefaultExports; -import lombok.Getter; - -public class ExportsInstance { - - @Getter private Node node; - - public ExportsInstance(Node node) { - this.node = node; - init(); - } - - private void init() { - // initialize jvm collector - DefaultExports.initialize(); - - // register collectors - CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; - // Job info detail - new JobMetricExports(this).register(collectorRegistry); - // Thread pool status - new JobThreadPoolStatusExports(this).register(collectorRegistry); - // Node metrics - new NodeMetricExports(this).register(collectorRegistry); - // Cluster metrics - new ClusterMetricExports(this).register(collectorRegistry); - } -} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java index ccc3f206bdd..f737fbf6c24 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java @@ -17,13 +17,37 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ClusterMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.NodeMetricExports; + import com.hazelcast.instance.impl.Node; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.hotspot.DefaultExports; public final class ExportsInstanceInitializer { + private static boolean initialized = false; + private ExportsInstanceInitializer() {} - public static void init(Node node) { - new ExportsInstance(node); + public static synchronized void init(Node node) { + if (!initialized) { + // initialize jvm collector + DefaultExports.initialize(); + + // register collectors + CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; + // Job info detail + new JobMetricExports(node).register(collectorRegistry); + // Thread pool status + new JobThreadPoolStatusExports(node).register(collectorRegistry); + // Node metrics + new NodeMetricExports(node).register(collectorRegistry); + // Cluster metrics + new ClusterMetricExports(node).register(collectorRegistry); + initialized = true; + } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java index df5cea73fa3..51b4cb448c7 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java @@ -18,9 +18,9 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.instance.impl.Node; import io.prometheus.client.GaugeMetricFamily; import java.net.UnknownHostException; @@ -30,8 +30,8 @@ public class ClusterMetricExports extends AbstractCollector { - public ClusterMetricExports(final ExportsInstance exportsInstance) { - super(exportsInstance); + public ClusterMetricExports(Node node) { + super(node); } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index 9f24f1ee399..7ae5574fb17 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -19,9 +19,9 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; +import com.hazelcast.instance.impl.Node; import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; @@ -29,8 +29,8 @@ public class JobMetricExports extends AbstractCollector { - public JobMetricExports(ExportsInstance exportsInstance) { - super(exportsInstance); + public JobMetricExports(Node node) { + super(node); } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index 88b3bb0e0bd..dcd3f7f31e4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -19,9 +19,9 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; +import com.hazelcast.instance.impl.Node; import io.prometheus.client.CounterMetricFamily; import io.prometheus.client.GaugeMetricFamily; @@ -33,8 +33,8 @@ public class JobThreadPoolStatusExports extends AbstractCollector { private static String HELP = "The %s of seatunnel coordinator job's executor cached thread pool"; - public JobThreadPoolStatusExports(ExportsInstance exportsInstance) { - super(exportsInstance); + public JobThreadPoolStatusExports(Node node) { + super(node); } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java index 25a5bfc42dd..c092ffd51b9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java @@ -18,8 +18,8 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; +import com.hazelcast.instance.impl.Node; import com.hazelcast.internal.jmx.InstanceMBean; import com.hazelcast.internal.jmx.PartitionServiceMBean; import io.prometheus.client.GaugeMetricFamily; @@ -29,8 +29,8 @@ public class NodeMetricExports extends AbstractCollector { - public NodeMetricExports(final ExportsInstance exportsInstance) { - super(exportsInstance); + public NodeMetricExports(Node node) { + super(node); } @Override From f15ba871f992c295844c00bee7aab5a59ebbdd96 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 8 Aug 2023 17:44:38 +0800 Subject: [PATCH 15/30] [zeta][telemetry] fix job thread metrics bug --- .../org/apache/seatunnel/engine/server/SeaTunnelServer.java | 5 +++++ .../metrics/exports/JobThreadPoolStatusExports.java | 4 +--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java index f75f5af8e4d..2486e2ba496 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.engine.server.execution.TaskGroupLocation; import org.apache.seatunnel.engine.server.service.slot.DefaultSlotService; import org.apache.seatunnel.engine.server.service.slot.SlotService; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; import com.hazelcast.internal.services.ManagedService; import com.hazelcast.internal.services.MembershipAwareService; @@ -253,4 +254,8 @@ private void printExecutionInfo() { coordinatorService.printJobDetailInfo(); } } + + public ThreadPoolStatus getThreadPoolStatusMetrics() { + return coordinatorService.getThreadPoolStatusMetrics(); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index dcd3f7f31e4..80ee89343e0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; -import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; @@ -40,11 +39,10 @@ public JobThreadPoolStatusExports(Node node) { @Override public List collect() { List mfs = new ArrayList(); - CoordinatorService coordinatorService = getCoordinatorService(); List labelValues = labelValues(localAddress()); - ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); + ThreadPoolStatus threadPoolStatusMetrics = getServer().getThreadPoolStatusMetrics(); List labelNames = clusterLabelNames(ADDRESS); GaugeMetricFamily activeCount = From 450f39d3e5189c9455394223d2c384af891dc9e8 Mon Sep 17 00:00:00 2001 From: kim-up Date: Thu, 29 Jun 2023 09:27:37 +0800 Subject: [PATCH 16/30] [Server][Metric] add prometheus --- .../seatunnel-engine-server/pom.xml | 26 +++++++++++++++++++ .../engine/server/SeaTunnelServerStarter.java | 9 ++++++- 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/seatunnel-engine/seatunnel-engine-server/pom.xml b/seatunnel-engine/seatunnel-engine-server/pom.xml index b31756674bf..00dfff2dbff 100644 --- a/seatunnel-engine/seatunnel-engine-server/pom.xml +++ b/seatunnel-engine/seatunnel-engine-server/pom.xml @@ -82,6 +82,32 @@ optional provided + + + + io.prometheus + simpleclient + 0.16.0 + + + + io.prometheus + simpleclient_hotspot + 0.16.0 + + + + io.prometheus + simpleclient_httpserver + 0.16.0 + + + + io.prometheus + simpleclient_pushgateway + 0.16.0 + + diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 4102ace4f25..04305b3073a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -17,6 +17,9 @@ package org.apache.seatunnel.engine.server; +import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.hotspot.DefaultExports; +import java.io.IOException; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; @@ -27,8 +30,12 @@ public class SeaTunnelServerStarter { - public static void main(String[] args) { + public static void main(String[] args) throws IOException { createHazelcastInstance(); + DefaultExports.initialize(); + HTTPServer httpServer = new HTTPServer.Builder() + .withPort(1234) + .build(); } public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { From 07f7307a9873b81bff7e6261008e30e66c882ebb Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 12 Jul 2023 18:16:38 +0800 Subject: [PATCH 17/30] [Server-core][telemetry] add telemetry core --- config/seatunnel.yaml | 6 +- docs/en/seatunnel-engine/telemetry.md | 45 +++++ .../engine/common/config/EngineConfig.java | 3 + .../YamlSeaTunnelDomConfigProcessor.java | 37 ++++ .../config/server/ServerConfigOptions.java | 24 +++ .../common/config/server/TelemetryConfig.java | 29 +++ .../config/server/TelemetryMetricConfig.java | 38 ++++ .../src/main/resources/seatunnel.yaml | 4 + .../engine/server/CoordinatorService.java | 173 +++++++++--------- .../engine/server/SeaTunnelServerStarter.java | 22 ++- .../telemetry/metrics/AbstractCollector.java | 63 +++++++ .../telemetry/metrics/ExportsInstance.java | 58 ++++++ .../metrics/ExportsInstanceFactory.java | 34 ++++ .../telemetry/metrics/entity/JobCounter.java | 37 ++++ .../metrics/entity/ThreadPoolStatus.java | 32 ++++ .../metrics/exports/JobMetricExports.java | 49 +++++ .../exports/ThreadPoolStatusExports.java | 48 +++++ 17 files changed, 610 insertions(+), 92 deletions(-) create mode 100644 docs/en/seatunnel-engine/telemetry.md create mode 100644 seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java create mode 100644 seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java diff --git a/config/seatunnel.yaml b/config/seatunnel.yaml index 7e496ca39ad..7936714c9df 100644 --- a/config/seatunnel.yaml +++ b/config/seatunnel.yaml @@ -34,4 +34,8 @@ seatunnel: plugin-config: namespace: /tmp/seatunnel/checkpoint_snapshot storage.type: hdfs - fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission \ No newline at end of file + fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission + telemetry: + metric: + http-port: 1025 + load-default-exports: false \ No newline at end of file diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md new file mode 100644 index 00000000000..4e994acacf9 --- /dev/null +++ b/docs/en/seatunnel-engine/telemetry.md @@ -0,0 +1,45 @@ +--- + +sidebar_position: 8 +------------------- + +# Telemetry + +Integrating `Open Telemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. + +You can configure the port exposed by the telemetry server in the `seatunnel.yaml` file. + +The following is an example declarative configuration. + +```yaml +seatunnel: + engine: + telemetry: + http-port: 1024 +``` + +*********************************************** + Job info detail +*********************************************** +createdJobCount : 0 +scheduledJobCount : 0 +runningJobCount : 0 +failingJobCount : 0 +failedJobCount : 0 +cancellingJobCount : 0 +canceledJobCount : 0 +finishedJobCount : 0 +restartingJobCount : 0 +suspendedJobCount : 0 +reconcilingJobCount : 0 + +*********************************************** + CoordinatorService Thread Pool Status +*********************************************** +activeCount : 0 +corePoolSize : 0 +maximumPoolSize : 2147483647 +poolSize : 0 +completedTaskCount : 0 +taskCount : 0 +*********************************************** diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java index edc18a0b15e..1f5731c201f 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/EngineConfig.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.engine.common.config.server.QueueType; import org.apache.seatunnel.engine.common.config.server.ServerConfigOptions; import org.apache.seatunnel.engine.common.config.server.SlotServiceConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryConfig; import org.apache.seatunnel.engine.common.config.server.ThreadShareMode; import lombok.Data; @@ -49,6 +50,8 @@ public class EngineConfig { private CheckpointConfig checkpointConfig = ServerConfigOptions.CHECKPOINT.defaultValue(); + private TelemetryConfig telemetryConfig = ServerConfigOptions.TELEMETRY.defaultValue(); + private QueueType queueType = ServerConfigOptions.QUEUE_TYPE.defaultValue(); public void setBackupCount(int newBackupCount) { diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java index a901fbb5e6a..88234c83fce 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java @@ -22,6 +22,8 @@ import org.apache.seatunnel.engine.common.config.server.QueueType; import org.apache.seatunnel.engine.common.config.server.ServerConfigOptions; import org.apache.seatunnel.engine.common.config.server.SlotServiceConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.common.config.server.ThreadShareMode; import org.w3c.dom.Node; @@ -131,6 +133,8 @@ private void parseEngineConfig(Node engineNode, SeaTunnelConfig config) { engineConfig.setSlotServiceConfig(parseSlotServiceConfig(node)); } else if (ServerConfigOptions.CHECKPOINT.key().equals(name)) { engineConfig.setCheckpointConfig(parseCheckpointConfig(node)); + } else if (ServerConfigOptions.TELEMETRY.key().equals(name)) { + engineConfig.setTelemetryConfig(parseTelemetryConfig(node)); } else { LOGGER.warning("Unrecognized element: " + name); } @@ -211,4 +215,37 @@ private Map parseCheckpointPluginConfig(Node checkpointPluginCon } return checkpointPluginConfig; } + + private TelemetryConfig parseTelemetryConfig(Node telemetryNode) { + TelemetryConfig metricConfig = new TelemetryConfig(); + for (Node node : childElements(telemetryNode)) { + String name = cleanNodeName(node); + if (ServerConfigOptions.TELEMETRY_METRIC.key().equals(name)) { + metricConfig.setMetric(parseTelemetryMetricConfig(node)); + } else { + LOGGER.warning("Unrecognized element: " + name); + } + } + + return metricConfig; + } + + private TelemetryMetricConfig parseTelemetryMetricConfig(Node metricNode) { + TelemetryMetricConfig metricConfig = new TelemetryMetricConfig(); + for (Node node : childElements(metricNode)) { + String name = cleanNodeName(node); + if (ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key().equals(name)) { + metricConfig.setHttpPort( + getIntegerValue( + ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key(), + getTextContent(node))); + } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.key().equals(name)) { + metricConfig.setLoadDefaultExports(getBooleanValue(getTextContent(node))); + } else { + LOGGER.warning("Unrecognized element: " + name); + } + } + + return metricConfig; + } } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index 2de8acad012..19fb671187c 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -145,4 +145,28 @@ public class ServerConfigOptions { .type(new TypeReference>() {}) .noDefaultValue() .withDescription("The checkpoint storage instance configuration."); + + public static final Option TELEMETRY_METRIC_HTTP_PORT = + Options.key("http-port") + .intType() + .defaultValue(1024) + .withDescription("The telemetry metric server's port."); + + public static final Option TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS = + Options.key("load-default-exports") + .booleanType() + .defaultValue(true) + .withDescription("Whether to load default jvm exports."); + + public static final Option TELEMETRY_METRIC = + Options.key("metric") + .type(new TypeReference() {}) + .defaultValue(new TelemetryMetricConfig()) + .withDescription("The telemetry metric configuration."); + + public static final Option TELEMETRY = + Options.key("telemetry") + .type(new TypeReference() {}) + .defaultValue(new TelemetryConfig()) + .withDescription("The telemetry configuration."); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java new file mode 100644 index 00000000000..45236e3660e --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.common.config.server; + +import lombok.Data; + +import java.io.Serializable; + +@Data +public class TelemetryConfig implements Serializable { + + private TelemetryMetricConfig metric = ServerConfigOptions.TELEMETRY_METRIC.defaultValue(); + +} diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java new file mode 100644 index 00000000000..41fa01a5f0d --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.common.config.server; + +import java.io.Serializable; +import lombok.Data; + +import static com.google.common.base.Preconditions.checkArgument; + +@Data +public class TelemetryMetricConfig implements Serializable { + + private int httpPort = ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.defaultValue(); + private boolean loadDefaultExports = + ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.defaultValue(); + + public void setHttpPort(int httpPort) { + checkArgument( + httpPort >= 0, + "The number of http's port failed checkpoints must be a natural number."); + this.httpPort = httpPort; + } +} diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index e5d92281da7..f3dd3c7179a 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -34,3 +34,7 @@ seatunnel: namespace: /tmp/seatunnel/checkpoint_snapshot/ storage.type: hdfs fs.defaultFS: file:///tmp/ + telemetry: + metric: + http-port: 1025 + load-default-exports: false diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index ced00d9e660..f2043ea305f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -48,6 +48,8 @@ import org.apache.seatunnel.engine.server.resourcemanager.ResourceManagerFactory; import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile; import org.apache.seatunnel.engine.server.task.operation.GetMetricsOperation; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; import org.apache.seatunnel.engine.server.utils.NodeEngineUtil; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -686,31 +688,54 @@ public void memberRemoved(MembershipServiceEvent event) { } public void printExecutionInfo() { - ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) executorService; - int activeCount = threadPoolExecutor.getActiveCount(); - int corePoolSize = threadPoolExecutor.getCorePoolSize(); - int maximumPoolSize = threadPoolExecutor.getMaximumPoolSize(); - int poolSize = threadPoolExecutor.getPoolSize(); - long completedTaskCount = threadPoolExecutor.getCompletedTaskCount(); - long taskCount = threadPoolExecutor.getTaskCount(); + ThreadPoolStatus threadPoolStatus = getThreadPoolStatusMetrics(); logger.info( StringFormatUtils.formatTable( "CoordinatorService Thread Pool Status", "activeCount", - activeCount, + threadPoolStatus.getActiveCount(), "corePoolSize", - corePoolSize, + threadPoolStatus.getCorePoolSize(), "maximumPoolSize", - maximumPoolSize, + threadPoolStatus.getMaximumPoolSize(), "poolSize", - poolSize, + threadPoolStatus.getPoolSize(), "completedTaskCount", - completedTaskCount, + threadPoolStatus.getCompletedTaskCount(), "taskCount", - taskCount)); + threadPoolStatus.getTaskCount())); } public void printJobDetailInfo() { + JobCounter jobCounter = getJobCountMetrics(); + logger.info( + StringFormatUtils.formatTable( + "Job info detail", + "createdJobCount", + jobCounter.getCreatedJobCount(), + "scheduledJobCount", + jobCounter.getScheduledJobCount(), + "runningJobCount", + jobCounter.getRunningJobCount(), + "failingJobCount", + jobCounter.getFailingJobCount(), + "failedJobCount", + jobCounter.getFailedJobCount(), + "cancellingJobCount", + jobCounter.getCancellingJobCount(), + "canceledJobCount", + jobCounter.getCanceledJobCount(), + "finishedJobCount", + jobCounter.getFailedJobCount(), + "restartingJobCount", + jobCounter.getRestartingJobCount(), + "suspendedJobCount", + jobCounter.getSuspendedJobCount(), + "reconcilingJobCount", + jobCounter.getReconcilingJobCount())); + } + + public JobCounter getJobCountMetrics() { AtomicLong createdJobCount = new AtomicLong(); AtomicLong scheduledJobCount = new AtomicLong(); AtomicLong runningJobCount = new AtomicLong(); @@ -725,76 +750,60 @@ public void printJobDetailInfo() { if (runningJobInfoIMap != null) { runningJobInfoIMap - .keySet() - .forEach( - jobId -> { - if (runningJobStateIMap.get(jobId) != null) { - JobStatus jobStatus = - (JobStatus) runningJobStateIMap.get(jobId); - switch (jobStatus) { - case CREATED: - createdJobCount.addAndGet(1); - break; - case SCHEDULED: - scheduledJobCount.addAndGet(1); - break; - case RUNNING: - runningJobCount.addAndGet(1); - break; - case FAILING: - failingJobCount.addAndGet(1); - break; - case FAILED: - failedJobCount.addAndGet(1); - break; - case CANCELLING: - cancellingJobCount.addAndGet(1); - break; - case CANCELED: - canceledJobCount.addAndGet(1); - break; - case FINISHED: - finishedJobCount.addAndGet(1); - break; - case RESTARTING: - restartingJobCount.addAndGet(1); - break; - case SUSPENDED: - suspendedJobCount.addAndGet(1); - break; - case RECONCILING: - reconcilingJobCount.addAndGet(1); - break; - default: - } - } - }); + .keySet() + .forEach( + jobId -> { + if (runningJobStateIMap.get(jobId) != null) { + JobStatus jobStatus = + (JobStatus) runningJobStateIMap.get(jobId); + switch (jobStatus) { + case CREATED: + createdJobCount.addAndGet(1); + break; + case SCHEDULED: + scheduledJobCount.addAndGet(1); + break; + case RUNNING: + runningJobCount.addAndGet(1); + break; + case FAILING: + failingJobCount.addAndGet(1); + break; + case FAILED: + failedJobCount.addAndGet(1); + break; + case CANCELLING: + cancellingJobCount.addAndGet(1); + break; + case CANCELED: + canceledJobCount.addAndGet(1); + break; + case FINISHED: + finishedJobCount.addAndGet(1); + break; + case RESTARTING: + restartingJobCount.addAndGet(1); + break; + case SUSPENDED: + suspendedJobCount.addAndGet(1); + break; + case RECONCILING: + reconcilingJobCount.addAndGet(1); + break; + default: + } + } + }); } + return new JobCounter(createdJobCount.longValue(), scheduledJobCount.longValue(), runningJobCount.longValue() + , failingJobCount.longValue(), failedJobCount.longValue(), cancellingJobCount.longValue(), canceledJobCount.longValue() + , finishedJobCount.longValue(), restartingJobCount.longValue(), suspendedJobCount.longValue(), reconcilingJobCount.longValue()); + } - logger.info( - StringFormatUtils.formatTable( - "Job info detail", - "createdJobCount", - createdJobCount, - "scheduledJobCount", - scheduledJobCount, - "runningJobCount", - runningJobCount, - "failingJobCount", - failingJobCount, - "failedJobCount", - failedJobCount, - "cancellingJobCount", - cancellingJobCount, - "canceledJobCount", - canceledJobCount, - "finishedJobCount", - finishedJobCount, - "restartingJobCount", - restartingJobCount, - "suspendedJobCount", - suspendedJobCount, - "reconcilingJobCount", - reconcilingJobCount)); + public ThreadPoolStatus getThreadPoolStatusMetrics(){ + ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) executorService; + return new ThreadPoolStatus(threadPoolExecutor.getActiveCount(), threadPoolExecutor.getCorePoolSize() + , threadPoolExecutor.getMaximumPoolSize(), threadPoolExecutor.getPoolSize() + , threadPoolExecutor.getCompletedTaskCount(), threadPoolExecutor.getTaskCount()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 04305b3073a..7e37a590d31 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -17,25 +17,22 @@ package org.apache.seatunnel.engine.server; -import io.prometheus.client.exporter.HTTPServer; -import io.prometheus.client.hotspot.DefaultExports; -import java.io.IOException; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstanceFactory; import com.hazelcast.instance.impl.HazelcastInstanceFactory; import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.instance.impl.HazelcastInstanceProxy; +import com.hazelcast.instance.impl.Node; import lombok.NonNull; +import java.io.IOException; + public class SeaTunnelServerStarter { public static void main(String[] args) throws IOException { createHazelcastInstance(); - DefaultExports.initialize(); - HTTPServer httpServer = new HTTPServer.Builder() - .withPort(1234) - .build(); } public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { @@ -55,8 +52,15 @@ public static HazelcastInstanceImpl createHazelcastInstance( .getOriginal(); } - public static HazelcastInstanceImpl createHazelcastInstance() { + public static HazelcastInstanceImpl createHazelcastInstance() throws IOException { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); - return createHazelcastInstance(seaTunnelConfig); + HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); + createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + return hazelcastInstance; + } + + private static void createTelemetryInstance( + @NonNull Node node, final SeaTunnelConfig seaTunnelConfig) throws IOException { + ExportsInstanceFactory.newExportsInstance(node, seaTunnelConfig); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java new file mode 100644 index 00000000000..3df2139731c --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics; + +import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.instance.impl.Node; +import com.hazelcast.logging.ILogger; +import io.prometheus.client.Collector; +import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.SeaTunnelServer; + +public abstract class AbstractCollector extends Collector { + + protected ExportsInstance exportsInstance; + + public AbstractCollector(final ExportsInstance exportsInstance) { + this.exportsInstance = exportsInstance; + } + + protected Node getNode() { + return exportsInstance.getNode(); + } + + protected ILogger getLogger(Class clazz) { + return getNode().getLogger(clazz); + } + + protected boolean isMaster() { + return getNode().isMaster(); + } + + protected MemberImpl getLocalMember() { + return getNode().nodeEngine.getLocalMember(); + } + + protected SeaTunnelServer getServer() { + return getNode().getNodeEngine().getService(SeaTunnelServer.SERVICE_NAME); + } + + protected CoordinatorService getCoordinatorService() { + if (isMaster()) { + return getServer().getCoordinatorService(); + } else { + throw new SeaTunnelEngineException("This is not a master node now."); + } + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java new file mode 100644 index 00000000000..ff8c38e0030 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics; + +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; + +import com.hazelcast.instance.impl.Node; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.hotspot.DefaultExports; + +import java.io.IOException; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; + +public class ExportsInstance { + + private Node node; + private TelemetryMetricConfig metricConfig; + + public ExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) throws IOException { + this.node = node; + this.metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); + start(); + } + + private void start() throws IOException { + if(metricConfig.isLoadDefaultExports()){ + DefaultExports.initialize(); + } + HTTPServer httpServer = + new HTTPServer.Builder().withPort(metricConfig.getHttpPort()).build(); + CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; + // + new JobMetricExports(this).register(collectorRegistry); + new ThreadPoolStatusExports(this).register(collectorRegistry); + } + + public Node getNode() { + return node; + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java new file mode 100644 index 00000000000..29a85469cec --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.telemetry.metrics; + +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; + +import com.hazelcast.instance.impl.Node; + +import java.io.IOException; + +public final class ExportsInstanceFactory { + + private ExportsInstanceFactory() {} + + public static ExportsInstance newExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) + throws IOException { + return new ExportsInstance(node, seaTunnelConfig); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java new file mode 100644 index 00000000000..09230e27467 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/JobCounter.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics.entity; + +import lombok.AllArgsConstructor; +import lombok.Data; + +@Data +@AllArgsConstructor +public class JobCounter { + private long createdJobCount; + private long scheduledJobCount; + private long runningJobCount; + private long failingJobCount; + private long failedJobCount; + private long cancellingJobCount; + private long canceledJobCount; + private long finishedJobCount; + private long restartingJobCount; + private long suspendedJobCount; + private long reconcilingJobCount; +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.java new file mode 100644 index 00000000000..ecbf341145d --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/entity/ThreadPoolStatus.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.seatunnel.engine.server.telemetry.metrics.entity; + +import lombok.AllArgsConstructor; +import lombok.Data; + +@Data +@AllArgsConstructor +public class ThreadPoolStatus { + private int activeCount; + private int corePoolSize; + private int maximumPoolSize; + private int poolSize; + private long completedTaskCount; + private long taskCount; +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java new file mode 100644 index 00000000000..e27d69141e8 --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics.exports; + +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; + +import io.prometheus.client.GaugeMetricFamily; + +import java.util.ArrayList; +import java.util.List; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; + +public class JobMetricExports extends AbstractCollector { + + public JobMetricExports(ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + // Only the master can get job metrics + if (isMaster()) { + CoordinatorService coordinatorService = getCoordinatorService(); + JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); + // TODO + mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); + } + return mfs; + } +} + diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java new file mode 100644 index 00000000000..4e070f3b22b --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.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.seatunnel.engine.server.telemetry.metrics.exports; + +import io.prometheus.client.GaugeMetricFamily; +import java.util.ArrayList; +import java.util.List; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; + +public class ThreadPoolStatusExports extends AbstractCollector { + + public ThreadPoolStatusExports(ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + // Only the master can get job metrics + if (isMaster()) { + CoordinatorService coordinatorService = getCoordinatorService(); + ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); + // TODO + mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); + } + return mfs; + } +} + From ec84c394dcfa9fbfd458043c7b7ef5710d987160 Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 12 Jul 2023 22:25:28 +0800 Subject: [PATCH 18/30] [Server-core][telemetry] add prometheus license and notice --- docs/en/seatunnel-engine/telemetry.md | 8 +- pom.xml | 21 ++++ seatunnel-dist/release-docs/LICENSE | 7 ++ seatunnel-dist/release-docs/NOTICE | 16 +++ .../YamlSeaTunnelDomConfigProcessor.java | 4 +- .../config/server/ServerConfigOptions.java | 16 +-- .../common/config/server/TelemetryConfig.java | 1 - .../config/server/TelemetryMetricConfig.java | 3 +- .../src/main/resources/seatunnel.yaml | 2 +- .../seatunnel-engine-server/pom.xml | 11 +- .../engine/server/CoordinatorService.java | 115 ++++++++++-------- .../telemetry/metrics/AbstractCollector.java | 12 +- .../telemetry/metrics/ExportsInstance.java | 6 +- .../metrics/exports/JobMetricExports.java | 39 +++++- .../exports/ThreadPoolStatusExports.java | 72 +++++++++-- tools/dependencies/known-dependencies.txt | 12 +- 16 files changed, 242 insertions(+), 103 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 4e994acacf9..9ad28f45c78 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -19,7 +19,8 @@ seatunnel: ``` *********************************************** - Job info detail + +Job info detail *********************************************** createdJobCount : 0 scheduledJobCount : 0 @@ -34,8 +35,11 @@ suspendedJobCount : 0 reconcilingJobCount : 0 *********************************************** - CoordinatorService Thread Pool Status + + CoordinatorService Thread Pool Status + *********************************************** + activeCount : 0 corePoolSize : 0 maximumPoolSize : 2147483647 diff --git a/pom.xml b/pom.xml index 224a339fc6e..006b9c2bdac 100644 --- a/pom.xml +++ b/pom.xml @@ -141,6 +141,8 @@ 4.2.0 true + + 0.16.0 @@ -447,6 +449,25 @@ provided + + + io.prometheus + simpleclient + ${prometheus.simpleclient.version} + + + + io.prometheus + simpleclient_hotspot + ${prometheus.simpleclient.version} + + + + io.prometheus + simpleclient_httpserver + ${prometheus.simpleclient.version} + + diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index 5d19a35a874..dca007e2f27 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -275,6 +275,13 @@ The text of each license is the standard Apache 2.0 license. (Apache-2.0) listenablefuture (com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava https://mvnrepository.com/artifact/com.google.guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava) (Apache-2.0) accessors-smart (com.google.guava:accessors-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/accessors-smart) (Apache-2.0) json-smart (net.minidev:json-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/json-smart) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient (io.prometheus:simpleclient:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Common (io.prometheus:simpleclient_common:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Hotspot (io.prometheus:simpleclient_hotspot:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Httpserver (io.prometheus:simpleclient_httpserver:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - Common (io.prometheus:simpleclient_tracer_common:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry (io.prometheus:simpleclient_tracer_otel:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry Agent (io.prometheus:simpleclient_tracer_otel_agent:0.16.0 - https://www.apache.org/hive-storage-api/) ======================================================================== MOZILLA PUBLIC LICENSE License diff --git a/seatunnel-dist/release-docs/NOTICE b/seatunnel-dist/release-docs/NOTICE index 4bed2d4be19..cbbf8cc6549 100644 --- a/seatunnel-dist/release-docs/NOTICE +++ b/seatunnel-dist/release-docs/NOTICE @@ -706,4 +706,20 @@ and updated within the WildFly project (https://github.com/wildfly/wildfly). The class org.apache.calcite.linq4j.tree.ConstantExpression contains code originating from the Calcite project (https://github.com/apache/calcite). +========================================================================= + +Prometheus NOTICE + +========================================================================= +Prometheus instrumentation library for JVM applications +Copyright 2012-2015 The Prometheus Authors + +This product includes software developed at +Boxever Ltd. (http://www.boxever.com/). + +This product includes software developed at +SoundCloud Ltd. (http://soundcloud.com/). + +This product includes software developed as part of the +Ocelli project by Netflix Inc. (https://github.com/Netflix/ocelli/). ========================================================================= \ No newline at end of file diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java index 88234c83fce..bf0d17f65ee 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java @@ -239,7 +239,9 @@ private TelemetryMetricConfig parseTelemetryMetricConfig(Node metricNode) { getIntegerValue( ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key(), getTextContent(node))); - } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.key().equals(name)) { + } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS + .key() + .equals(name)) { metricConfig.setLoadDefaultExports(getBooleanValue(getTextContent(node))); } else { LOGGER.warning("Unrecognized element: " + name); diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index 19fb671187c..12426d0bc14 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -159,14 +159,14 @@ public class ServerConfigOptions { .withDescription("Whether to load default jvm exports."); public static final Option TELEMETRY_METRIC = - Options.key("metric") - .type(new TypeReference() {}) - .defaultValue(new TelemetryMetricConfig()) - .withDescription("The telemetry metric configuration."); + Options.key("metric") + .type(new TypeReference() {}) + .defaultValue(new TelemetryMetricConfig()) + .withDescription("The telemetry metric configuration."); public static final Option TELEMETRY = - Options.key("telemetry") - .type(new TypeReference() {}) - .defaultValue(new TelemetryConfig()) - .withDescription("The telemetry configuration."); + Options.key("telemetry") + .type(new TypeReference() {}) + .defaultValue(new TelemetryConfig()) + .withDescription("The telemetry configuration."); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java index 45236e3660e..c3e603eea4c 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryConfig.java @@ -25,5 +25,4 @@ public class TelemetryConfig implements Serializable { private TelemetryMetricConfig metric = ServerConfigOptions.TELEMETRY_METRIC.defaultValue(); - } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java index 41fa01a5f0d..c4d10cdd0b5 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java @@ -17,9 +17,10 @@ package org.apache.seatunnel.engine.common.config.server; -import java.io.Serializable; import lombok.Data; +import java.io.Serializable; + import static com.google.common.base.Preconditions.checkArgument; @Data diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index f3dd3c7179a..da4b9857ef4 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -37,4 +37,4 @@ seatunnel: telemetry: metric: http-port: 1025 - load-default-exports: false + load-default-exports: true diff --git a/seatunnel-engine/seatunnel-engine-server/pom.xml b/seatunnel-engine/seatunnel-engine-server/pom.xml index 00dfff2dbff..608bc173880 100644 --- a/seatunnel-engine/seatunnel-engine-server/pom.xml +++ b/seatunnel-engine/seatunnel-engine-server/pom.xml @@ -83,29 +83,20 @@ provided - + io.prometheus simpleclient - 0.16.0 io.prometheus simpleclient_hotspot - 0.16.0 io.prometheus simpleclient_httpserver - 0.16.0 - - - - io.prometheus - simpleclient_pushgateway - 0.16.0 diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java index f2043ea305f..38f197e604e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java @@ -750,60 +750,73 @@ public JobCounter getJobCountMetrics() { if (runningJobInfoIMap != null) { runningJobInfoIMap - .keySet() - .forEach( - jobId -> { - if (runningJobStateIMap.get(jobId) != null) { - JobStatus jobStatus = - (JobStatus) runningJobStateIMap.get(jobId); - switch (jobStatus) { - case CREATED: - createdJobCount.addAndGet(1); - break; - case SCHEDULED: - scheduledJobCount.addAndGet(1); - break; - case RUNNING: - runningJobCount.addAndGet(1); - break; - case FAILING: - failingJobCount.addAndGet(1); - break; - case FAILED: - failedJobCount.addAndGet(1); - break; - case CANCELLING: - cancellingJobCount.addAndGet(1); - break; - case CANCELED: - canceledJobCount.addAndGet(1); - break; - case FINISHED: - finishedJobCount.addAndGet(1); - break; - case RESTARTING: - restartingJobCount.addAndGet(1); - break; - case SUSPENDED: - suspendedJobCount.addAndGet(1); - break; - case RECONCILING: - reconcilingJobCount.addAndGet(1); - break; - default: - } - } - }); + .keySet() + .forEach( + jobId -> { + if (runningJobStateIMap.get(jobId) != null) { + JobStatus jobStatus = + (JobStatus) runningJobStateIMap.get(jobId); + switch (jobStatus) { + case CREATED: + createdJobCount.addAndGet(1); + break; + case SCHEDULED: + scheduledJobCount.addAndGet(1); + break; + case RUNNING: + runningJobCount.addAndGet(1); + break; + case FAILING: + failingJobCount.addAndGet(1); + break; + case FAILED: + failedJobCount.addAndGet(1); + break; + case CANCELLING: + cancellingJobCount.addAndGet(1); + break; + case CANCELED: + canceledJobCount.addAndGet(1); + break; + case FINISHED: + finishedJobCount.addAndGet(1); + break; + case RESTARTING: + restartingJobCount.addAndGet(1); + break; + case SUSPENDED: + suspendedJobCount.addAndGet(1); + break; + case RECONCILING: + reconcilingJobCount.addAndGet(1); + break; + default: + } + } + }); } - return new JobCounter(createdJobCount.longValue(), scheduledJobCount.longValue(), runningJobCount.longValue() - , failingJobCount.longValue(), failedJobCount.longValue(), cancellingJobCount.longValue(), canceledJobCount.longValue() - , finishedJobCount.longValue(), restartingJobCount.longValue(), suspendedJobCount.longValue(), reconcilingJobCount.longValue()); + return new JobCounter( + createdJobCount.longValue(), + scheduledJobCount.longValue(), + runningJobCount.longValue(), + failingJobCount.longValue(), + failedJobCount.longValue(), + cancellingJobCount.longValue(), + canceledJobCount.longValue(), + finishedJobCount.longValue(), + restartingJobCount.longValue(), + suspendedJobCount.longValue(), + reconcilingJobCount.longValue()); } - public ThreadPoolStatus getThreadPoolStatusMetrics(){ + public ThreadPoolStatus getThreadPoolStatusMetrics() { ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) executorService; - return new ThreadPoolStatus(threadPoolExecutor.getActiveCount(), threadPoolExecutor.getCorePoolSize() - , threadPoolExecutor.getMaximumPoolSize(), threadPoolExecutor.getPoolSize() - , threadPoolExecutor.getCompletedTaskCount(), threadPoolExecutor.getTaskCount()); + return new ThreadPoolStatus( + threadPoolExecutor.getActiveCount(), + threadPoolExecutor.getCorePoolSize(), + threadPoolExecutor.getMaximumPoolSize(), + threadPoolExecutor.getPoolSize(), + threadPoolExecutor.getCompletedTaskCount(), + threadPoolExecutor.getTaskCount()); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java index 3df2139731c..6547a2708b4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -17,13 +17,13 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; +import org.apache.seatunnel.engine.server.CoordinatorService; +import org.apache.seatunnel.engine.server.SeaTunnelServer; + import com.hazelcast.cluster.impl.MemberImpl; import com.hazelcast.instance.impl.Node; import com.hazelcast.logging.ILogger; import io.prometheus.client.Collector; -import org.apache.seatunnel.engine.common.exception.SeaTunnelEngineException; -import org.apache.seatunnel.engine.server.CoordinatorService; -import org.apache.seatunnel.engine.server.SeaTunnelServer; public abstract class AbstractCollector extends Collector { @@ -54,10 +54,6 @@ protected SeaTunnelServer getServer() { } protected CoordinatorService getCoordinatorService() { - if (isMaster()) { - return getServer().getCoordinatorService(); - } else { - throw new SeaTunnelEngineException("This is not a master node now."); - } + return getServer().getCoordinatorService(); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java index ff8c38e0030..cfff6d012db 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -19,6 +19,8 @@ import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; import com.hazelcast.instance.impl.Node; import io.prometheus.client.CollectorRegistry; @@ -26,8 +28,6 @@ import io.prometheus.client.hotspot.DefaultExports; import java.io.IOException; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; public class ExportsInstance { @@ -41,7 +41,7 @@ public ExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) throws IOExce } private void start() throws IOException { - if(metricConfig.isLoadDefaultExports()){ + if (metricConfig.isLoadDefaultExports()) { DefaultExports.initialize(); } HTTPServer httpServer = diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index e27d69141e8..625380ea107 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -20,12 +20,13 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; public class JobMetricExports extends AbstractCollector { @@ -40,10 +41,40 @@ public List collect() { if (isMaster()) { CoordinatorService coordinatorService = getCoordinatorService(); JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); - // TODO - mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "st_job_count", + "The job count of seatunnel cluster ", + Collections.singletonList("type")); + + metricFamily.addMetric( + Collections.singletonList("canceled"), jobCountMetrics.getCanceledJobCount()); + metricFamily.addMetric( + Collections.singletonList("cancelling"), + jobCountMetrics.getCancellingJobCount()); + metricFamily.addMetric( + Collections.singletonList("created"), jobCountMetrics.getCreatedJobCount()); + metricFamily.addMetric( + Collections.singletonList("failed"), jobCountMetrics.getFailedJobCount()); + metricFamily.addMetric( + Collections.singletonList("failing"), jobCountMetrics.getFailingJobCount()); + metricFamily.addMetric( + Collections.singletonList("finished"), jobCountMetrics.getFinishedJobCount()); + metricFamily.addMetric( + Collections.singletonList("reconciling"), + jobCountMetrics.getReconcilingJobCount()); + metricFamily.addMetric( + Collections.singletonList("restarting"), + jobCountMetrics.getRestartingJobCount()); + metricFamily.addMetric( + Collections.singletonList("running"), jobCountMetrics.getRunningJobCount()); + metricFamily.addMetric( + Collections.singletonList("scheduled"), jobCountMetrics.getScheduledJobCount()); + metricFamily.addMetric( + Collections.singletonList("suspended"), jobCountMetrics.getSuspendedJobCount()); + + mfs.add(metricFamily); } return mfs; } } - diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java index 4e070f3b22b..1f3c9944e80 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java @@ -17,17 +17,22 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; -import io.prometheus.client.GaugeMetricFamily; -import java.util.ArrayList; -import java.util.List; import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; -import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; +import io.prometheus.client.CounterMetricFamily; +import io.prometheus.client.GaugeMetricFamily; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + public class ThreadPoolStatusExports extends AbstractCollector { + private static String HELP = "The %s of seatunnel coordinator executors cachedThreadPool"; + public ThreadPoolStatusExports(ExportsInstance exportsInstance) { super(exportsInstance); } @@ -35,14 +40,57 @@ public ThreadPoolStatusExports(ExportsInstance exportsInstance) { @Override public List collect() { List mfs = new ArrayList(); - // Only the master can get job metrics - if (isMaster()) { - CoordinatorService coordinatorService = getCoordinatorService(); - ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); - // TODO - mfs.add(new GaugeMetricFamily("jobMetric", "jobMetric", 1)); - } + CoordinatorService coordinatorService = getCoordinatorService(); + + String address = getNode().getNodeEngine().getLocalMember().getAddress().toString(); + List labelValues = Collections.singletonList(address); + + ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); + List labelNames = Collections.singletonList("address"); + GaugeMetricFamily activeCount = + new GaugeMetricFamily( + "st_thread_pool_activeCount", + String.format(HELP, "activeCount"), + labelNames); + activeCount.addMetric(labelValues, threadPoolStatusMetrics.getActiveCount()); + mfs.add(activeCount); + + CounterMetricFamily completedTask = + new CounterMetricFamily( + "st_thread_pool_completedTask", + String.format(HELP, "completedTask"), + labelNames); + completedTask.addMetric(labelValues, threadPoolStatusMetrics.getCompletedTaskCount()); + mfs.add(completedTask); + + GaugeMetricFamily corePoolSize = + new GaugeMetricFamily( + "st_thread_pool_corePoolSize", + String.format(HELP, "activeCount"), + labelNames); + corePoolSize.addMetric(labelValues, threadPoolStatusMetrics.getCorePoolSize()); + mfs.add(corePoolSize); + + GaugeMetricFamily maximumPoolSize = + new GaugeMetricFamily( + "st_thread_pool_maximumPoolSize", + String.format(HELP, "maximumPoolSize"), + labelNames); + maximumPoolSize.addMetric(labelValues, threadPoolStatusMetrics.getMaximumPoolSize()); + mfs.add(maximumPoolSize); + + GaugeMetricFamily poolSize = + new GaugeMetricFamily( + "st_thread_pool_poolSize", String.format(HELP, "poolSize"), labelNames); + poolSize.addMetric(labelValues, threadPoolStatusMetrics.getPoolSize()); + mfs.add(poolSize); + + CounterMetricFamily taskCount = + new CounterMetricFamily( + "st_thread_pool_task", String.format(HELP, "taskCount"), labelNames); + taskCount.addMetric(labelValues, threadPoolStatusMetrics.getTaskCount()); + mfs.add(taskCount); + return mfs; } } - diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 11e5c57b8af..47e60953926 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -34,4 +34,14 @@ j2objc-annotations-1.1.jar jsr305-1.3.9.jar jsr305-3.0.0.jar jsr305-3.0.2.jar -listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar \ No newline at end of file +listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar +accessors-smart-2.4.7.jar +asm-9.1.jar +json-smart-2.4.7.jar +simpleclient-0.16.0.jar +simpleclient_common-0.16.0.jar +simpleclient_hotspot-0.16.0.jar +simpleclient_httpserver-0.16.0.jar +simpleclient_tracer_common-0.16.0.jar +simpleclient_tracer_otel-0.16.0.jar +simpleclient_tracer_otel_agent-0.16.0.jar From e26ef7a3c96724884ea78ba8e115bbc9d7fcb539 Mon Sep 17 00:00:00 2001 From: kim-up Date: Thu, 13 Jul 2023 12:11:39 +0800 Subject: [PATCH 19/30] [Server-core][telemetry] add e2e test --- config/seatunnel.yaml | 2 +- docs/en/seatunnel-engine/telemetry.md | 33 +--- .../seatunnel/engine/e2e/TelemetryApiIT.java | 141 ++++++++++++++++++ .../config/server/ServerConfigOptions.java | 2 +- .../src/main/resources/seatunnel.yaml | 2 +- .../engine/server/SeaTunnelServerStarter.java | 2 +- 6 files changed, 147 insertions(+), 35 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java diff --git a/config/seatunnel.yaml b/config/seatunnel.yaml index 7936714c9df..67c42868e31 100644 --- a/config/seatunnel.yaml +++ b/config/seatunnel.yaml @@ -37,5 +37,5 @@ seatunnel: fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission telemetry: metric: - http-port: 1025 + http-port: 9090 load-default-exports: false \ No newline at end of file diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 9ad28f45c78..a1b63c187ac 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -5,7 +5,7 @@ sidebar_position: 8 # Telemetry -Integrating `Open Telemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. +Integrating `OpenTelemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. You can configure the port exposed by the telemetry server in the `seatunnel.yaml` file. @@ -15,35 +15,6 @@ The following is an example declarative configuration. seatunnel: engine: telemetry: - http-port: 1024 + http-port: 9090 ``` -*********************************************** - -Job info detail -*********************************************** -createdJobCount : 0 -scheduledJobCount : 0 -runningJobCount : 0 -failingJobCount : 0 -failedJobCount : 0 -cancellingJobCount : 0 -canceledJobCount : 0 -finishedJobCount : 0 -restartingJobCount : 0 -suspendedJobCount : 0 -reconcilingJobCount : 0 - -*********************************************** - - CoordinatorService Thread Pool Status - -*********************************************** - -activeCount : 0 -corePoolSize : 0 -maximumPoolSize : 2147483647 -poolSize : 0 -completedTaskCount : 0 -taskCount : 0 -*********************************************** diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java new file mode 100644 index 00000000000..c68da5faeac --- /dev/null +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.e2e; + +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.seatunnel.common.config.Common; +import org.apache.seatunnel.common.config.DeployMode; +import org.apache.seatunnel.engine.client.SeaTunnelClient; +import org.apache.seatunnel.engine.client.job.ClientJobProxy; +import org.apache.seatunnel.engine.client.job.JobExecutionEnvironment; +import org.apache.seatunnel.engine.common.config.ConfigProvider; +import org.apache.seatunnel.engine.common.config.JobConfig; +import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; +import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; +import org.apache.seatunnel.engine.core.job.JobStatus; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; +import org.apache.seatunnel.engine.server.rest.RestConstant; +import org.awaitility.Awaitility; +import org.hamcrest.Matcher; +import org.hamcrest.core.StringContains; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static io.restassured.RestAssured.given; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.matchesRegex; + +@Slf4j +public class TelemetryApiIT { + + private static final String HOST = "http://localhost:"; + + private static ClientJobProxy clientJobProxy; + + private static HazelcastInstanceImpl hazelcastInstance; + + private static TelemetryMetricConfig metricConfig; + + @BeforeAll + static void beforeClass() throws Exception { + String testClusterName = TestUtils.getClusterName("TelemetryApiIT"); + SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); + seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName); + // get TelemetryMetricConfig + metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); + hazelcastInstance = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); + // createTelemetryInstance + SeaTunnelServerStarter.createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + Common.setDeployMode(DeployMode.CLIENT); + String filePath = TestUtils.getResource("stream_fakesource_to_file.conf"); + JobConfig jobConfig = new JobConfig(); + jobConfig.setName("fake_to_file"); + + ClientConfig clientConfig = ConfigProvider.locateAndGetClientConfig(); + clientConfig.setClusterName(testClusterName); + SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig); + JobExecutionEnvironment jobExecutionEnv = + engineClient.createExecutionContext(filePath, jobConfig); + + clientJobProxy = jobExecutionEnv.execute(); + + Awaitility.await() + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + Assertions.assertEquals( + JobStatus.RUNNING, clientJobProxy.getJobStatus())); + } + + @Test + public void testGetMetrics() { + given().get( + HOST + + metricConfig.getHttpPort() + + "/metrics") + .then() + .statusCode(200) + // Use regular expressions to verify whether the response body is the indicator data of Prometheus + // Metric data is usually multi-line, use newlines for validation + .body(matchesRegex("(?s)^.*# HELP.*# TYPE.*$")) + // Verify that the response body contains a specific metric + // JVM metrics + .body(containsString("jvm_threads")) + .body(containsString("jvm_memory_pool")) + .body(containsString("jvm_gc")) + .body(containsString("jvm_info")) + .body(containsString("jvm_memory_bytes")) + .body(containsString("jvm_classes")) + .body(containsString("jvm_buffer_pool")) + .body(containsString("process_start")) + // Thread pool metrics + .body(containsString("st_thread_pool_activeCount{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) + .body(containsString("st_thread_pool_poolSize{address=\"[localhost]:5801\",}")) + .body(containsString("st_thread_pool_task_total{address=\"[localhost]:5801\",}")) + // Job count metrics + .body(containsString("st_job_count{type=\"canceled\",} 0.0")) + .body(containsString("st_job_count{type=\"cancelling\",} 0.0")) + .body(containsString("st_job_count{type=\"created\",} 0.0")) + .body(containsString("st_job_count{type=\"failed\",} 0.0")) + .body(containsString("st_job_count{type=\"failing\",} 0.0")) + .body(containsString("st_job_count{type=\"finished\",} 0.0")) + .body(containsString("st_job_count{type=\"reconciling\",} 0.0")) + .body(containsString("st_job_count{type=\"restarting\",} 0.0")) + // Running job count is 1 + .body(containsString("st_job_count{type=\"running\",} 1.0")) + .body(containsString("st_job_count{type=\"scheduled\",} 0.0")) + .body(containsString("st_job_count{type=\"suspended\",} 0.0")); + } + + @AfterAll + static void afterClass() { + if (hazelcastInstance != null) { + hazelcastInstance.shutdown(); + } + } +} diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index 12426d0bc14..5a3e9e55024 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -149,7 +149,7 @@ public class ServerConfigOptions { public static final Option TELEMETRY_METRIC_HTTP_PORT = Options.key("http-port") .intType() - .defaultValue(1024) + .defaultValue(9090) .withDescription("The telemetry metric server's port."); public static final Option TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS = diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index da4b9857ef4..9ceb2ed68a3 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -36,5 +36,5 @@ seatunnel: fs.defaultFS: file:///tmp/ telemetry: metric: - http-port: 1025 + http-port: 9090 load-default-exports: true diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 7e37a590d31..91da8fc362f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -59,7 +59,7 @@ public static HazelcastInstanceImpl createHazelcastInstance() throws IOException return hazelcastInstance; } - private static void createTelemetryInstance( + public static void createTelemetryInstance( @NonNull Node node, final SeaTunnelConfig seaTunnelConfig) throws IOException { ExportsInstanceFactory.newExportsInstance(node, seaTunnelConfig); } From 39011576b72a257ac156ad659cea161db993173f Mon Sep 17 00:00:00 2001 From: kim-up Date: Fri, 14 Jul 2023 23:27:19 +0800 Subject: [PATCH 20/30] [Server-core][telemetry] update telemetry docs --- docs/en/seatunnel-engine/telemetry.md | 69 ++++++++++- release-note.md | 1 + .../seatunnel/core/starter/SeaTunnel.java | 5 +- .../core/starter/command/Command.java | 4 +- .../core/starter/flink/SeaTunnelFlink.java | 4 +- .../core/starter/flink/SeaTunnelFlink.java | 4 +- .../core/starter/spark/SeaTunnelSpark.java | 4 +- .../core/starter/spark/SeaTunnelSpark.java | 4 +- .../starter/seatunnel/SeaTunnelClient.java | 4 +- .../starter/seatunnel/SeaTunnelServer.java | 4 +- .../command/ServerExecuteCommand.java | 11 +- .../seatunnel/engine/e2e/TelemetryApiIT.java | 110 +++++++++--------- .../src/test/resources/hazelcast.yaml | 2 + .../engine/server/SeaTunnelServerStarter.java | 7 +- .../telemetry/metrics/ExportsInstance.java | 7 +- .../metrics/exports/JobMetricExports.java | 2 +- ...s.java => JobThreadPoolStatusExports.java} | 19 +-- .../engine/server/CoordinatorServiceTest.java | 8 +- .../engine/SeaTunnelEngineExample.java | 8 +- .../engine/SeaTunnelEngineServerExample.java | 4 +- .../example/flink/v2/SeaTunnelApiExample.java | 3 +- .../example/spark/v2/ExampleUtils.java | 3 +- .../example/spark/v2/SeaTunnelApiExample.java | 4 +- 23 files changed, 192 insertions(+), 99 deletions(-) rename seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/{ThreadPoolStatusExports.java => JobThreadPoolStatusExports.java} (84%) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index a1b63c187ac..c880c26f8bb 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -7,7 +7,7 @@ sidebar_position: 8 Integrating `OpenTelemetry` through `Prometheus-exports` can better seamlessly connect to related monitoring platforms such as Prometheus and Grafana, improving the ability to monitor and alarm of the Seatunnel cluster. -You can configure the port exposed by the telemetry server in the `seatunnel.yaml` file. +You can configure telemetry's configurations in the `seatunnel.yaml` file. The following is an example declarative configuration. @@ -15,6 +15,71 @@ The following is an example declarative configuration. seatunnel: engine: telemetry: - http-port: 9090 + http-port: 9090 # The port exposed by the telemetry server, default is 9090. + load-default-exports: true # Whether to load default jvm exports, default is true. ``` +## Metrics + +Available metrics include the following categories. + +### Thread Pool Status + +| MetricName | Type | Labels | +|-------------------------------------|---------|----------------------------------------------------------------------| +| job_thread_pool_activeCount | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_corePoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_maximumPoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_poolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_completedTask_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | +| job_thread_pool_task_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | + +### Job info detail + +| MetricName | Type | Labels | +|------------|-------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| job_count | Gauge | **type**, the type of job, including: "canceled" "cancelling" "created" "failed" "failing" "finished" "reconciling" "restarting" "running" "scheduled" "suspended" | + +### JVM Metrics + +| MetricName | Type | Labels | +|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------| +| jvm_threads_current | Gauge | - | +| jvm_threads_daemon | Gauge | - | +| jvm_threads_daemon | Gauge | - | +| jvm_threads_peak | Gauge | - | +| jvm_threads_started_total | Counter | - | +| jvm_threads_deadlocked | Gauge | - | +| jvm_threads_deadlocked_monitor | Gauge | - | +| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | +| jvm_classes_currently_loaded | Gauge | - | +| jvm_classes_loaded_total | Counter | - | +| jvm_classes_unloaded_total | Counter | - | +| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | +| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | +| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | +| process_cpu_seconds_total | Counter | - | +| process_start_time_seconds | Gauge | - | +| process_open_fds | Gauge | - | +| process_max_fds | Gauge | - | +| jvm_memory_objects_pending_finalization | Gauge | - | +| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | +| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | +| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | +| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | +| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | +| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | +| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | +| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | +| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | + +### 实现方案 + diff --git a/release-note.md b/release-note.md index d647bdbad93..0ec37a63032 100644 --- a/release-note.md +++ b/release-note.md @@ -175,6 +175,7 @@ - [Zeta] Reduce the frequency of fetching data from imap (#4851) - [Zeta] Add OSS support for Imap storage to cluster-mode type (#4683) - [Zeta] Improve local mode startup request ports (#4660) +- [Zeta] Support exposing jobCount metrics, jobThreadPool metrics and JVM metrics by prometheus exporter (#5070) ## Docs diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java index e003bf2d206..22afb8fc9dc 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/SeaTunnel.java @@ -26,6 +26,8 @@ import lombok.extern.slf4j.Slf4j; +import java.io.IOException; + @Slf4j public class SeaTunnel { @@ -35,7 +37,8 @@ public class SeaTunnel { * @param command commandArgs * @param commandType */ - public static void run(Command command) throws CommandException { + public static void run(Command command) + throws CommandException, IOException { try { command.execute(); } catch (ConfigRuntimeException e) { diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java index 1d510ab984b..e419c89b46b 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/command/Command.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.core.starter.exception.CommandExecuteException; import org.apache.seatunnel.core.starter.exception.ConfigCheckException; +import java.io.IOException; + /** * Command interface, only has one method {@link Command#execute()}, used to execute the command * @@ -28,5 +30,5 @@ @FunctionalInterface public interface Command { - void execute() throws CommandExecuteException, ConfigCheckException; + void execute() throws CommandExecuteException, ConfigCheckException, IOException; } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java index 8d1b434801d..460b4f2874d 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelFlink { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java index 1595da686a7..e7baa0e7811 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelFlink { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java index ca7b2ed4be1..2de9c4ba930 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java @@ -23,9 +23,11 @@ import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelSpark { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { SparkCommandArgs sparkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java index 9b3fde6fd10..85cf2005d51 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java @@ -23,9 +23,11 @@ import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelSpark { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { SparkCommandArgs sparkCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java index 7a37fd340c3..3a755f808ab 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelClient.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ClientCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelClient { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { ClientCommandArgs clientCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java index 96a3e32e51a..ae3284fc010 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/SeaTunnelServer.java @@ -23,8 +23,10 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ServerCommandArgs; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import java.io.IOException; + public class SeaTunnelServer { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { ServerCommandArgs serverCommandArgs = CommandLineUtils.parse( args, diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java index aead1f8a964..e5d11fc3b5c 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java @@ -21,11 +21,11 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ServerCommandArgs; import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.server.SeaTunnelNodeContext; +import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; import org.apache.commons.lang3.StringUtils; -import com.hazelcast.instance.impl.HazelcastInstanceFactory; +import java.io.IOException; /** This command is used to execute the SeaTunnel engine job by SeaTunnel API. */ public class ServerExecuteCommand implements Command { @@ -37,14 +37,11 @@ public ServerExecuteCommand(ServerCommandArgs serverCommandArgs) { } @Override - public void execute() { + public void execute() throws IOException { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); if (StringUtils.isNotEmpty(serverCommandArgs.getClusterName())) { seaTunnelConfig.getHazelcastConfig().setClusterName(serverCommandArgs.getClusterName()); } - HazelcastInstanceFactory.newHazelcastInstance( - seaTunnelConfig.getHazelcastConfig(), - Thread.currentThread().getName(), - new SeaTunnelNodeContext(seaTunnelConfig)); + SeaTunnelServerStarter.createHazelcastInstance(); } } diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java index c68da5faeac..c3b2129c5c3 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -17,10 +17,6 @@ package org.apache.seatunnel.engine.e2e; -import com.hazelcast.client.config.ClientConfig; -import com.hazelcast.instance.impl.HazelcastInstanceImpl; -import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.common.config.DeployMode; import org.apache.seatunnel.engine.client.SeaTunnelClient; @@ -32,19 +28,21 @@ import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; -import org.apache.seatunnel.engine.server.rest.RestConstant; + import org.awaitility.Awaitility; -import org.hamcrest.Matcher; -import org.hamcrest.core.StringContains; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import com.hazelcast.client.config.ClientConfig; +import com.hazelcast.instance.impl.HazelcastInstanceImpl; +import lombok.extern.slf4j.Slf4j; + +import java.util.concurrent.TimeUnit; + import static io.restassured.RestAssured.given; import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.matchesRegex; @Slf4j @@ -77,59 +75,61 @@ static void beforeClass() throws Exception { clientConfig.setClusterName(testClusterName); SeaTunnelClient engineClient = new SeaTunnelClient(clientConfig); JobExecutionEnvironment jobExecutionEnv = - engineClient.createExecutionContext(filePath, jobConfig); + engineClient.createExecutionContext(filePath, jobConfig); clientJobProxy = jobExecutionEnv.execute(); Awaitility.await() - .atMost(2, TimeUnit.MINUTES) - .untilAsserted( - () -> - Assertions.assertEquals( - JobStatus.RUNNING, clientJobProxy.getJobStatus())); + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> + Assertions.assertEquals( + JobStatus.RUNNING, clientJobProxy.getJobStatus())); } @Test - public void testGetMetrics() { - given().get( - HOST - + metricConfig.getHttpPort() - + "/metrics") - .then() - .statusCode(200) - // Use regular expressions to verify whether the response body is the indicator data of Prometheus - // Metric data is usually multi-line, use newlines for validation - .body(matchesRegex("(?s)^.*# HELP.*# TYPE.*$")) - // Verify that the response body contains a specific metric - // JVM metrics - .body(containsString("jvm_threads")) - .body(containsString("jvm_memory_pool")) - .body(containsString("jvm_gc")) - .body(containsString("jvm_info")) - .body(containsString("jvm_memory_bytes")) - .body(containsString("jvm_classes")) - .body(containsString("jvm_buffer_pool")) - .body(containsString("process_start")) - // Thread pool metrics - .body(containsString("st_thread_pool_activeCount{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) - .body(containsString("st_thread_pool_poolSize{address=\"[localhost]:5801\",}")) - .body(containsString("st_thread_pool_task_total{address=\"[localhost]:5801\",}")) - // Job count metrics - .body(containsString("st_job_count{type=\"canceled\",} 0.0")) - .body(containsString("st_job_count{type=\"cancelling\",} 0.0")) - .body(containsString("st_job_count{type=\"created\",} 0.0")) - .body(containsString("st_job_count{type=\"failed\",} 0.0")) - .body(containsString("st_job_count{type=\"failing\",} 0.0")) - .body(containsString("st_job_count{type=\"finished\",} 0.0")) - .body(containsString("st_job_count{type=\"reconciling\",} 0.0")) - .body(containsString("st_job_count{type=\"restarting\",} 0.0")) - // Running job count is 1 - .body(containsString("st_job_count{type=\"running\",} 1.0")) - .body(containsString("st_job_count{type=\"scheduled\",} 0.0")) - .body(containsString("st_job_count{type=\"suspended\",} 0.0")); + public void testGetMetrics() throws InterruptedException { + given().get(HOST + metricConfig.getHttpPort() + "/metrics") + .then() + .statusCode(200) + // Use regular expressions to verify whether the response body is the indicator data + // of Prometheus + // Metric data is usually multi-line, use newlines for validation + .body(matchesRegex("(?s)^.*# HELP.*# TYPE.*$")) + // Verify that the response body contains a specific metric + // JVM metrics + .body(containsString("jvm_threads")) + .body(containsString("jvm_memory_pool")) + .body(containsString("jvm_gc")) + .body(containsString("jvm_info")) + .body(containsString("jvm_memory_bytes")) + .body(containsString("jvm_classes")) + .body(containsString("jvm_buffer_pool")) + .body(containsString("process_start")) + // Job thread pool metrics + .body(containsString("job_thread_pool_activeCount{address=\"[localhost]:5801\",}")) + .body( + containsString( + "job_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) + .body(containsString("job_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) + .body( + containsString( + "job_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) + .body(containsString("job_thread_pool_poolSize{address=\"[localhost]:5801\",}")) + .body(containsString("job_thread_pool_task_total{address=\"[localhost]:5801\",}")) + // Job count metrics + .body(containsString("job_count{type=\"canceled\",} 0.0")) + .body(containsString("job_count{type=\"cancelling\",} 0.0")) + .body(containsString("job_count{type=\"created\",} 0.0")) + .body(containsString("job_count{type=\"failed\",} 0.0")) + .body(containsString("job_count{type=\"failing\",} 0.0")) + .body(containsString("job_count{type=\"finished\",} 0.0")) + .body(containsString("job_count{type=\"reconciling\",} 0.0")) + .body(containsString("job_count{type=\"restarting\",} 0.0")) + // Running job count is 1 + .body(containsString("job_count{type=\"running\",} 1.0")) + .body(containsString("job_count{type=\"scheduled\",} 0.0")) + .body(containsString("job_count{type=\"suspended\",} 0.0")); } @AfterAll diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml index c28e9c94d5c..e1048ab83f9 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml @@ -15,6 +15,7 @@ # limitations under the License. # + hazelcast: cluster-name: seatunnel network: @@ -41,3 +42,4 @@ hazelcast: hazelcast.slow.operation.detector.stacktrace.logging.enabled: true hazelcast.logging.type: log4j2 hazelcast.operation.generic.thread.count: 200 +# hazelcast.jet.enabled: true diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 91da8fc362f..51e18cfea5f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -35,10 +35,13 @@ public static void main(String[] args) throws IOException { createHazelcastInstance(); } - public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { + public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) + throws IOException { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); seaTunnelConfig.getHazelcastConfig().setClusterName(clusterName); - return createHazelcastInstance(seaTunnelConfig); + HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); + createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + return hazelcastInstance; } public static HazelcastInstanceImpl createHazelcastInstance( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java index cfff6d012db..5f636dd6c45 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -20,7 +20,7 @@ import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ThreadPoolStatusExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; import com.hazelcast.instance.impl.Node; import io.prometheus.client.CollectorRegistry; @@ -47,9 +47,10 @@ private void start() throws IOException { HTTPServer httpServer = new HTTPServer.Builder().withPort(metricConfig.getHttpPort()).build(); CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; - // + // Job info detail new JobMetricExports(this).register(collectorRegistry); - new ThreadPoolStatusExports(this).register(collectorRegistry); + // Thread pool status + new JobThreadPoolStatusExports(this).register(collectorRegistry); } public Node getNode() { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index 625380ea107..471a32c5aab 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -43,7 +43,7 @@ public List collect() { JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); GaugeMetricFamily metricFamily = new GaugeMetricFamily( - "st_job_count", + "job_count", "The job count of seatunnel cluster ", Collections.singletonList("type")); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java similarity index 84% rename from seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java rename to seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index 1f3c9944e80..c38c34be42d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -29,11 +29,12 @@ import java.util.Collections; import java.util.List; -public class ThreadPoolStatusExports extends AbstractCollector { +public class JobThreadPoolStatusExports extends AbstractCollector { - private static String HELP = "The %s of seatunnel coordinator executors cachedThreadPool"; + private static String HELP = + "The %s of seatunnel coordinator job's executor cached thread pool"; - public ThreadPoolStatusExports(ExportsInstance exportsInstance) { + public JobThreadPoolStatusExports(ExportsInstance exportsInstance) { super(exportsInstance); } @@ -49,7 +50,7 @@ public List collect() { List labelNames = Collections.singletonList("address"); GaugeMetricFamily activeCount = new GaugeMetricFamily( - "st_thread_pool_activeCount", + "job_thread_pool_activeCount", String.format(HELP, "activeCount"), labelNames); activeCount.addMetric(labelValues, threadPoolStatusMetrics.getActiveCount()); @@ -57,7 +58,7 @@ public List collect() { CounterMetricFamily completedTask = new CounterMetricFamily( - "st_thread_pool_completedTask", + "job_thread_pool_completedTask", String.format(HELP, "completedTask"), labelNames); completedTask.addMetric(labelValues, threadPoolStatusMetrics.getCompletedTaskCount()); @@ -65,7 +66,7 @@ public List collect() { GaugeMetricFamily corePoolSize = new GaugeMetricFamily( - "st_thread_pool_corePoolSize", + "job_thread_pool_corePoolSize", String.format(HELP, "activeCount"), labelNames); corePoolSize.addMetric(labelValues, threadPoolStatusMetrics.getCorePoolSize()); @@ -73,7 +74,7 @@ public List collect() { GaugeMetricFamily maximumPoolSize = new GaugeMetricFamily( - "st_thread_pool_maximumPoolSize", + "job_thread_pool_maximumPoolSize", String.format(HELP, "maximumPoolSize"), labelNames); maximumPoolSize.addMetric(labelValues, threadPoolStatusMetrics.getMaximumPoolSize()); @@ -81,13 +82,13 @@ public List collect() { GaugeMetricFamily poolSize = new GaugeMetricFamily( - "st_thread_pool_poolSize", String.format(HELP, "poolSize"), labelNames); + "job_thread_pool_poolSize", String.format(HELP, "poolSize"), labelNames); poolSize.addMetric(labelValues, threadPoolStatusMetrics.getPoolSize()); mfs.add(poolSize); CounterMetricFamily taskCount = new CounterMetricFamily( - "st_thread_pool_task", String.format(HELP, "taskCount"), labelNames); + "job_thread_pool_task", String.format(HELP, "taskCount"), labelNames); taskCount.addMetric(labelValues, threadPoolStatusMetrics.getTaskCount()); mfs.add(taskCount); diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java index 3cb224e69e0..ba65ded1194 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java @@ -31,9 +31,9 @@ import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.internal.serialization.Data; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.net.MalformedURLException; import java.util.Collections; import java.util.concurrent.TimeUnit; @@ -41,7 +41,7 @@ public class CoordinatorServiceTest { @Test - public void testMasterNodeActive() { + public void testMasterNodeActive() throws IOException { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName("CoordinatorServiceTest_testMasterNodeActive")); @@ -85,7 +85,7 @@ public void testMasterNodeActive() { @SuppressWarnings("checkstyle:RegexpSingleline") @Test public void testClearCoordinatorService() - throws MalformedURLException, NoSuchMethodException, InvocationTargetException, + throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { HazelcastInstanceImpl coordinatorServiceTest = SeaTunnelServerStarter.createHazelcastInstance( @@ -141,7 +141,7 @@ public void testClearCoordinatorService() @Test @Disabled("disabled because we can not know") - public void testJobRestoreWhenMasterNodeSwitch() throws InterruptedException { + public void testJobRestoreWhenMasterNodeSwitch() throws InterruptedException, IOException { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName( diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index 2a7c25e0830..b43dc0cf0e3 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.core.starter.seatunnel.args.ClientCommandArgs; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; @@ -30,8 +31,9 @@ public class SeaTunnelEngineExample { public static void main(String[] args) - throws FileNotFoundException, URISyntaxException, CommandException { - String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; + throws IOException, URISyntaxException, CommandException { + String configurePath = + args.length > 0 ? args[0] : "/examples/stream_fakesource_to_file.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); @@ -39,7 +41,7 @@ public static void main(String[] args) clientCommandArgs.setJobName(Paths.get(configFile).getFileName().toString()); // Change Execution Mode to CLUSTER to use client mode, before do this, you should start // SeaTunnelEngineServerExample - clientCommandArgs.setMasterType(MasterType.LOCAL); + clientCommandArgs.setMasterType(MasterType.CLUSTER); SeaTunnel.run(clientCommandArgs.buildCommand()); } diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java index df539ea6e05..2bb1ed69c4b 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineServerExample.java @@ -21,8 +21,10 @@ import org.apache.seatunnel.core.starter.exception.CommandException; import org.apache.seatunnel.core.starter.seatunnel.args.ServerCommandArgs; +import java.io.IOException; + public class SeaTunnelEngineServerExample { - public static void main(String[] args) throws CommandException { + public static void main(String[] args) throws CommandException, IOException { ServerCommandArgs serverCommandArgs = new ServerCommandArgs(); SeaTunnel.run(serverCommandArgs.buildCommand()); } diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java index 553c1963622..f6b1c6a7941 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; @@ -29,7 +30,7 @@ public class SeaTunnelApiExample { public static void main(String[] args) - throws FileNotFoundException, URISyntaxException, CommandException { + throws IOException, URISyntaxException, CommandException { String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); FlinkCommandArgs flinkCommandArgs = new FlinkCommandArgs(); diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java index 2c7ddc5de69..0bbfff04356 100644 --- a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java +++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/ExampleUtils.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; @@ -30,7 +31,7 @@ public class ExampleUtils { public static void builder(String configurePath) - throws FileNotFoundException, URISyntaxException, CommandException { + throws IOException, URISyntaxException, CommandException { String configFile = getTestConfigFile(configurePath); SparkCommandArgs sparkCommandArgs = new SparkCommandArgs(); sparkCommandArgs.setConfigFile(configFile); diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java index 870ec64895a..b0a12dc3fc4 100644 --- a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java +++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java @@ -19,13 +19,13 @@ import org.apache.seatunnel.core.starter.exception.CommandException; -import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URISyntaxException; public class SeaTunnelApiExample { public static void main(String[] args) - throws FileNotFoundException, URISyntaxException, CommandException { + throws IOException, URISyntaxException, CommandException { String configurePath = args.length > 0 ? args[0] : "/examples/spark.batch.conf"; ExampleUtils.builder(configurePath); } From 1c5bfc604ab1176e1480307aefba4d30dea4e9c2 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 18:54:30 +0800 Subject: [PATCH 21/30] [Server-core][telemetry] improving code and docs --- config/seatunnel.yaml | 3 +- .../seatunnel-engine/grafana/dashboard.json | 1904 +++++++++++++++++ .../seatunnel-engine/openmetrics/metrics.txt | 280 +++ .../seatunnel-engine/prometheus/metrics.txt | 275 +++ docs/en/seatunnel-engine/telemetry.md | 167 +- .../command/ServerExecuteCommand.java | 4 +- seatunnel-dist/release-docs/LICENSE | 14 +- .../seatunnel/engine/e2e/JobExecutionIT.java | 2 +- .../seatunnel/engine/e2e/TelemetryApiIT.java | 441 +++- .../src/test/resources/hazelcast.yaml | 1 - .../YamlSeaTunnelDomConfigProcessor.java | 11 +- .../config/server/ServerConfigOptions.java | 12 +- .../config/server/TelemetryMetricConfig.java | 13 +- .../src/main/resources/seatunnel.yaml | 3 +- .../engine/server/NodeExtension.java | 4 + .../engine/server/SeaTunnelServerStarter.java | 31 +- .../engine/server/rest/RestConstant.java | 3 + .../rest/RestHttpGetCommandProcessor.java | 34 + .../telemetry/metrics/AbstractCollector.java | 62 + .../telemetry/metrics/ExportsInstance.java | 35 +- ...y.java => ExportsInstanceInitializer.java} | 13 +- .../metrics/exports/ClusterMetricExports.java | 94 + .../metrics/exports/JobMetricExports.java | 37 +- .../exports/JobThreadPoolStatusExports.java | 9 +- .../metrics/exports/NodeMetricExports.java | 407 ++++ .../engine/server/CoordinatorServiceTest.java | 8 +- 26 files changed, 3670 insertions(+), 197 deletions(-) create mode 100644 docs/en/seatunnel-engine/grafana/dashboard.json create mode 100644 docs/en/seatunnel-engine/openmetrics/metrics.txt create mode 100644 docs/en/seatunnel-engine/prometheus/metrics.txt rename seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/{ExportsInstanceFactory.java => ExportsInstanceInitializer.java} (71%) create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java create mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java diff --git a/config/seatunnel.yaml b/config/seatunnel.yaml index 67c42868e31..104126437d2 100644 --- a/config/seatunnel.yaml +++ b/config/seatunnel.yaml @@ -37,5 +37,4 @@ seatunnel: fs.defaultFS: file:///tmp/ # Ensure that the directory has written permission telemetry: metric: - http-port: 9090 - load-default-exports: false \ No newline at end of file + enabled: true \ No newline at end of file diff --git a/docs/en/seatunnel-engine/grafana/dashboard.json b/docs/en/seatunnel-engine/grafana/dashboard.json new file mode 100644 index 00000000000..a0457631055 --- /dev/null +++ b/docs/en/seatunnel-engine/grafana/dashboard.json @@ -0,0 +1,1904 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 0, + "id": 1, + "links": [], + "liveNow": false, + "panels": [ + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "semi-dark-orange", + "value": null + }, + { + "color": "red", + "value": 100000 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 17, + "options": { + "colorMode": "background", + "graphMode": "none", + "justifyMode": "center", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "8.3.3", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "node_count", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "Total Node Count", + "type": "stat" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 18, + "options": { + "colorMode": "background", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "text": { + "titleSize": 1 + }, + "textMode": "auto" + }, + "pluginVersion": "8.3.3", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "sum(node_state)", + "interval": "", + "legendFormat": "", + "refId": "A" + } + ], + "title": "UP Node Count", + "type": "stat" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 22, + "panels": [], + "title": "Hazelcast Partition", + "type": "row" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 5 + }, + "id": 32, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_partitionCount", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "partitionCount", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 5 + }, + "id": 33, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_activePartition", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "activePartition", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 12 + }, + "id": 34, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "pluginVersion": "8.3.3", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_isClusterSafe", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "isClusterSafe", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 12 + }, + "id": 35, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_partition_isLocalMemberSafe", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "isLocalMemberSafe", + "type": "timeseries" + }, + { + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 19 + }, + "id": 20, + "title": "Hazelcast Executor", + "type": "row" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 24, + "x": 0, + "y": 20 + }, + "id": 24, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_executedCount", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "executedCount", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 26 + }, + "id": 26, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_isTerminated", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "isTerminated", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 26 + }, + "id": 25, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_isShutdown", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "isShutdown", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 34 + }, + "id": 28, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_poolSize", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "poolSize", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 34 + }, + "id": 27, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_maxPoolSize", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "maxPoolSize", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 42 + }, + "id": 30, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_queueRemainingCapacity", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "queueRemainingCapacity", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 42 + }, + "id": 29, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "hazelcast_executor_queueSize", + "interval": "", + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" + } + ], + "title": "queueSize", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 50 + }, + "id": 7, + "panels": [], + "title": "System", + "type": "row" + }, + { + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 51 + }, + "id": 9, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "100 * (1 - rate(process_cpu_seconds_total[15s]))", + "interval": "", + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "title": "Cpu Usage", + "type": "timeseries" + }, + { + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 51 + }, + "id": 10, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "100 * (jvm_memory_bytes_used{area=\"heap\"} / jvm_memory_bytes_max{area=\"heap\"})", + "interval": "", + "legendFormat": "{{instance}}-{{area}}", + "refId": "A" + } + ], + "title": "Heap Memory Usage", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 59 + }, + "id": 12, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "round(delta(jvm_gc_collection_seconds_count[15s]))", + "interval": "", + "legendFormat": "{{instance}}-{{gc}}", + "refId": "A" + } + ], + "title": "GC Count", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 59 + }, + "id": 13, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "delta(jvm_gc_collection_seconds_sum[15s]) * 1000", + "interval": "", + "legendFormat": "{{instance}}-{{gc}}", + "refId": "A" + } + ], + "title": "GC Cost Time", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 67 + }, + "id": 14, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "right" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_current", + "interval": "", + "legendFormat": "{{instance}}-current", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_daemon", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-daemon", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_peak", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-peak", + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "jvm_threads_deadlocked", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-deadlocked", + "refId": "D" + } + ], + "title": "Jvm Threads", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 75 + }, + "id": 5, + "panels": [], + "title": "Job", + "type": "row" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 24, + "x": 0, + "y": 76 + }, + "id": 2, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "sum(job_count) by (type) ", + "interval": "", + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "title": "Job Count", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 82 + }, + "id": 3, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_activeCount", + "interval": "", + "legendFormat": "{{instance}}-activeCount", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_corePoolSize", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-corePoolSize", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_maximumPoolSize", + "hide": true, + "interval": "", + "legendFormat": "{{instance}}-maximumPoolSize", + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_poolSize", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-poolSize", + "refId": "D" + } + ], + "title": "Job Thread Pool", + "type": "timeseries" + }, + { + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 82 + }, + "id": 15, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom" + }, + "tooltip": { + "mode": "single" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_completedTask_total", + "interval": "", + "legendFormat": "{{instance}}-completedTaskTotal", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "jUi2yaj4k" + }, + "exemplar": true, + "expr": "job_thread_pool_task_total", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-taskTotal", + "refId": "B" + } + ], + "title": "Job Thread Pool Total", + "type": "timeseries" + } + ], + "refresh": "5s", + "schemaVersion": 34, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-15m", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "Seatunnel Cluster", + "uid": "BDfcs-CVz", + "version": 11, + "weekStart": "" +} \ No newline at end of file diff --git a/docs/en/seatunnel-engine/openmetrics/metrics.txt b/docs/en/seatunnel-engine/openmetrics/metrics.txt new file mode 100644 index 00000000000..ea1855fb113 --- /dev/null +++ b/docs/en/seatunnel-engine/openmetrics/metrics.txt @@ -0,0 +1,280 @@ +# TYPE job_count gauge +# HELP job_count All job counts of seatunnel cluster +job_count{cluster="seatunnel",type="canceled"} 0.0 +job_count{cluster="seatunnel",type="cancelling"} 0.0 +job_count{cluster="seatunnel",type="created"} 0.0 +job_count{cluster="seatunnel",type="failed"} 0.0 +job_count{cluster="seatunnel",type="failing"} 0.0 +job_count{cluster="seatunnel",type="finished"} 0.0 +job_count{cluster="seatunnel",type="reconciling"} 0.0 +job_count{cluster="seatunnel",type="restarting"} 0.0 +job_count{cluster="seatunnel",type="running"} 0.0 +job_count{cluster="seatunnel",type="scheduled"} 0.0 +job_count{cluster="seatunnel",type="suspended"} 0.0 +# TYPE process_cpu_seconds counter +# HELP process_cpu_seconds Total user and system CPU time spent in seconds. +process_cpu_seconds_total 15.74875 +# TYPE process_start_time_seconds gauge +# HELP process_start_time_seconds Start time of the process since unix epoch in seconds. +process_start_time_seconds 1.689671779061E9 +# TYPE process_open_fds gauge +# HELP process_open_fds Number of open file descriptors. +process_open_fds 112.0 +# TYPE process_max_fds gauge +# HELP process_max_fds Maximum number of open file descriptors. +process_max_fds 10240.0 +# TYPE cluster_info gauge +# HELP cluster_info Cluster info +cluster_info{cluster="seatunnel",hazelcastVersion="5.1",master="127.0.0.1:5801"} 1.0 +# TYPE cluster_time gauge +# HELP cluster_time Cluster start time +cluster_time{cluster="seatunnel",hazelcastVersion="5.1"} 1.68967187357E12 +# TYPE node_count gauge +# HELP node_count Cluster node total count +node_count{cluster="seatunnel"} 1.0 +# TYPE jvm_threads_current gauge +# HELP jvm_threads_current Current thread count of a JVM +jvm_threads_current 105.0 +# TYPE jvm_threads_daemon gauge +# HELP jvm_threads_daemon Daemon thread count of a JVM +jvm_threads_daemon 7.0 +# TYPE jvm_threads_peak gauge +# HELP jvm_threads_peak Peak thread count of a JVM +jvm_threads_peak 118.0 +# TYPE jvm_threads_started counter +# HELP jvm_threads_started Started thread count of a JVM +jvm_threads_started_total 122.0 +# TYPE jvm_threads_deadlocked gauge +# HELP jvm_threads_deadlocked Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers +jvm_threads_deadlocked 0.0 +# TYPE jvm_threads_deadlocked_monitor gauge +# HELP jvm_threads_deadlocked_monitor Cycles of JVM-threads that are in deadlock waiting to acquire object monitors +jvm_threads_deadlocked_monitor 0.0 +# TYPE jvm_threads_state gauge +# HELP jvm_threads_state Current count of threads by state +jvm_threads_state{state="NEW"} 0.0 +jvm_threads_state{state="TERMINATED"} 0.0 +jvm_threads_state{state="RUNNABLE"} 12.0 +jvm_threads_state{state="BLOCKED"} 0.0 +jvm_threads_state{state="WAITING"} 76.0 +jvm_threads_state{state="TIMED_WAITING"} 17.0 +jvm_threads_state{state="UNKNOWN"} 0.0 +# TYPE jvm_classes_currently_loaded gauge +# HELP jvm_classes_currently_loaded The number of classes that are currently loaded in the JVM +jvm_classes_currently_loaded 8317.0 +# TYPE jvm_classes_loaded counter +# HELP jvm_classes_loaded The total number of classes that have been loaded since the JVM has started execution +jvm_classes_loaded_total 8317.0 +# TYPE jvm_classes_unloaded counter +# HELP jvm_classes_unloaded The total number of classes that have been unloaded since the JVM has started execution +jvm_classes_unloaded_total 0.0 +# TYPE node_state gauge +# HELP node_state Whether is up of seatunnel node +node_state{cluster="seatunnel",address="127.0.0.1:5801"} 1.0 +# TYPE hazelcast_executor_executedCount gauge +# HELP hazelcast_executor_executedCount The hazelcast executor executedCount of seatunnel cluster node +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 136.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 1601.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_isShutdown gauge +# HELP hazelcast_executor_isShutdown The hazelcast executor isShutdown of seatunnel cluster node +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_isTerminated gauge +# HELP hazelcast_executor_isTerminated The hazelcast executor isTerminated of seatunnel cluster node +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_maxPoolSize gauge +# HELP hazelcast_executor_maxPoolSize The hazelcast executor maxPoolSize of seatunnel cluster node +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 160.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 8.0 +# TYPE hazelcast_executor_poolSize gauge +# HELP hazelcast_executor_poolSize The hazelcast executor poolSize of seatunnel cluster node +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_executor_queueRemainingCapacity gauge +# HELP hazelcast_executor_queueRemainingCapacity The hazelcast executor queueRemainingCapacity of seatunnel cluster +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 2.147483647E9 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 2.147483647E9 +# TYPE hazelcast_executor_queueSize gauge +# HELP hazelcast_executor_queueSize The hazelcast executor queueSize of seatunnel cluster node +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="async"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="client"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientBlocking"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="clientQuery"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="io"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="offloadable"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="scheduled"} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="127.0.0.1:5801",type="system"} 0.0 +# TYPE hazelcast_partition_partitionCount gauge +# HELP hazelcast_partition_partitionCount The partitionCount of seatunnel cluster node +hazelcast_partition_partitionCount{cluster="seatunnel",address="127.0.0.1:5801"} 271.0 +# TYPE hazelcast_partition_activePartition gauge +# HELP hazelcast_partition_activePartition The activePartition of seatunnel cluster node +hazelcast_partition_activePartition{cluster="seatunnel",address="127.0.0.1:5801"} 271.0 +# TYPE hazelcast_partition_isClusterSafe gauge +# HELP hazelcast_partition_isClusterSafe Whether is cluster safe of partition +hazelcast_partition_isClusterSafe{cluster="seatunnel",address="127.0.0.1:5801"} 1.0 +# TYPE hazelcast_partition_isLocalMemberSafe gauge +# HELP hazelcast_partition_isLocalMemberSafe Whether is local member safe of partition +hazelcast_partition_isLocalMemberSafe{cluster="seatunnel",address="127.0.0.1:5801"} 1.0 +# TYPE jvm_memory_pool_allocated_bytes counter +# HELP jvm_memory_pool_allocated_bytes Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously. +jvm_memory_pool_allocated_bytes_total{pool="Code Cache"} 8118976.0 +jvm_memory_pool_allocated_bytes_created{pool="Code Cache"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="PS Eden Space"} 2.39174704E8 +jvm_memory_pool_allocated_bytes_created{pool="PS Eden Space"} 1.689671786515E9 +jvm_memory_pool_allocated_bytes_total{pool="PS Old Gen"} 1.8928464E7 +jvm_memory_pool_allocated_bytes_created{pool="PS Old Gen"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="PS Survivor Space"} 9945064.0 +jvm_memory_pool_allocated_bytes_created{pool="PS Survivor Space"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="Compressed Class Space"} 4591800.0 +jvm_memory_pool_allocated_bytes_created{pool="Compressed Class Space"} 1.689671786522E9 +jvm_memory_pool_allocated_bytes_total{pool="Metaspace"} 3.2616592E7 +jvm_memory_pool_allocated_bytes_created{pool="Metaspace"} 1.689671786522E9 +# TYPE jvm_buffer_pool_used_bytes gauge +# HELP jvm_buffer_pool_used_bytes Used bytes of a given JVM buffer pool. +jvm_buffer_pool_used_bytes{pool="direct"} 412746.0 +jvm_buffer_pool_used_bytes{pool="mapped"} 0.0 +# TYPE jvm_buffer_pool_capacity_bytes gauge +# HELP jvm_buffer_pool_capacity_bytes Bytes capacity of a given JVM buffer pool. +jvm_buffer_pool_capacity_bytes{pool="direct"} 412743.0 +jvm_buffer_pool_capacity_bytes{pool="mapped"} 0.0 +# TYPE jvm_buffer_pool_used_buffers gauge +# HELP jvm_buffer_pool_used_buffers Used buffers of a given JVM buffer pool. +jvm_buffer_pool_used_buffers{pool="direct"} 7.0 +jvm_buffer_pool_used_buffers{pool="mapped"} 0.0 +# TYPE job_thread_pool_activeCount gauge +# HELP job_thread_pool_activeCount The activeCount of seatunnel coordinator job's executor cached thread pool +job_thread_pool_activeCount{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_completedTask counter +# HELP job_thread_pool_completedTask The completedTask of seatunnel coordinator job's executor cached thread pool +job_thread_pool_completedTask_total{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_corePoolSize gauge +# HELP job_thread_pool_corePoolSize The corePoolSize of seatunnel coordinator job's executor cached thread pool +job_thread_pool_corePoolSize{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_maximumPoolSize gauge +# HELP job_thread_pool_maximumPoolSize The maximumPoolSize of seatunnel coordinator job's executor cached thread pool +job_thread_pool_maximumPoolSize{cluster="seatunnel",address="127.0.0.1:5801"} 2.147483647E9 +# TYPE job_thread_pool_poolSize gauge +# HELP job_thread_pool_poolSize The poolSize of seatunnel coordinator job's executor cached thread pool +job_thread_pool_poolSize{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE job_thread_pool_task counter +# HELP job_thread_pool_task The taskCount of seatunnel coordinator job's executor cached thread pool +job_thread_pool_task_total{cluster="seatunnel",address="127.0.0.1:5801"} 0.0 +# TYPE jvm_gc_collection_seconds summary +# HELP jvm_gc_collection_seconds Time spent in a given JVM garbage collector in seconds. +jvm_gc_collection_seconds_count{gc="PS Scavenge"} 6.0 +jvm_gc_collection_seconds_sum{gc="PS Scavenge"} 0.161 +jvm_gc_collection_seconds_count{gc="PS MarkSweep"} 2.0 +jvm_gc_collection_seconds_sum{gc="PS MarkSweep"} 0.068 +# TYPE jvm_memory_objects_pending_finalization gauge +# HELP jvm_memory_objects_pending_finalization The number of objects waiting in the finalizer queue. +jvm_memory_objects_pending_finalization 0.0 +# TYPE jvm_memory_bytes_used gauge +# HELP jvm_memory_bytes_used Used bytes of a given JVM memory area. +jvm_memory_bytes_used{area="heap"} 2.72221784E8 +jvm_memory_bytes_used{area="nonheap"} 5.6619408E7 +# TYPE jvm_memory_bytes_committed gauge +# HELP jvm_memory_bytes_committed Committed (bytes) of a given JVM memory area. +jvm_memory_bytes_committed{area="heap"} 6.13941248E8 +jvm_memory_bytes_committed{area="nonheap"} 6.2537728E7 +# TYPE jvm_memory_bytes_max gauge +# HELP jvm_memory_bytes_max Max (bytes) of a given JVM memory area. +jvm_memory_bytes_max{area="heap"} 3.817865216E9 +jvm_memory_bytes_max{area="nonheap"} -1.0 +# TYPE jvm_memory_bytes_init gauge +# HELP jvm_memory_bytes_init Initial bytes of a given JVM memory area. +jvm_memory_bytes_init{area="heap"} 2.68435456E8 +jvm_memory_bytes_init{area="nonheap"} 2555904.0 +# TYPE jvm_memory_pool_bytes_used gauge +# HELP jvm_memory_pool_bytes_used Used bytes of a given JVM memory pool. +jvm_memory_pool_bytes_used{pool="Code Cache"} 1.0232064E7 +jvm_memory_pool_bytes_used{pool="Metaspace"} 4.0859424E7 +jvm_memory_pool_bytes_used{pool="Compressed Class Space"} 5527920.0 +jvm_memory_pool_bytes_used{pool="PS Eden Space"} 2.5329332E8 +jvm_memory_pool_bytes_used{pool="PS Survivor Space"} 0.0 +jvm_memory_pool_bytes_used{pool="PS Old Gen"} 1.8928464E7 +# TYPE jvm_memory_pool_bytes_committed gauge +# HELP jvm_memory_pool_bytes_committed Committed bytes of a given JVM memory pool. +jvm_memory_pool_bytes_committed{pool="Code Cache"} 1.081344E7 +jvm_memory_pool_bytes_committed{pool="Metaspace"} 4.52608E7 +jvm_memory_pool_bytes_committed{pool="Compressed Class Space"} 6463488.0 +jvm_memory_pool_bytes_committed{pool="PS Eden Space"} 4.90733568E8 +jvm_memory_pool_bytes_committed{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_bytes_committed{pool="PS Old Gen"} 1.13246208E8 +# TYPE jvm_memory_pool_bytes_max gauge +# HELP jvm_memory_pool_bytes_max Max bytes of a given JVM memory pool. +jvm_memory_pool_bytes_max{pool="Code Cache"} 2.5165824E8 +jvm_memory_pool_bytes_max{pool="Metaspace"} -1.0 +jvm_memory_pool_bytes_max{pool="Compressed Class Space"} 1.073741824E9 +jvm_memory_pool_bytes_max{pool="PS Eden Space"} 1.40771328E9 +jvm_memory_pool_bytes_max{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_bytes_max{pool="PS Old Gen"} 2.863661056E9 +# TYPE jvm_memory_pool_bytes_init gauge +# HELP jvm_memory_pool_bytes_init Initial bytes of a given JVM memory pool. +jvm_memory_pool_bytes_init{pool="Code Cache"} 2555904.0 +jvm_memory_pool_bytes_init{pool="Metaspace"} 0.0 +jvm_memory_pool_bytes_init{pool="Compressed Class Space"} 0.0 +jvm_memory_pool_bytes_init{pool="PS Eden Space"} 6.7108864E7 +jvm_memory_pool_bytes_init{pool="PS Survivor Space"} 1.1010048E7 +jvm_memory_pool_bytes_init{pool="PS Old Gen"} 1.79306496E8 +# TYPE jvm_memory_pool_collection_used_bytes gauge +# HELP jvm_memory_pool_collection_used_bytes Used bytes after last collection of a given JVM memory pool. +jvm_memory_pool_collection_used_bytes{pool="PS Eden Space"} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Survivor Space"} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Old Gen"} 1.8928464E7 +# TYPE jvm_memory_pool_collection_committed_bytes gauge +# HELP jvm_memory_pool_collection_committed_bytes Committed after last collection bytes of a given JVM memory pool. +jvm_memory_pool_collection_committed_bytes{pool="PS Eden Space"} 4.90733568E8 +jvm_memory_pool_collection_committed_bytes{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_collection_committed_bytes{pool="PS Old Gen"} 1.13246208E8 +# TYPE jvm_memory_pool_collection_max_bytes gauge +# HELP jvm_memory_pool_collection_max_bytes Max bytes after last collection of a given JVM memory pool. +jvm_memory_pool_collection_max_bytes{pool="PS Eden Space"} 1.40771328E9 +jvm_memory_pool_collection_max_bytes{pool="PS Survivor Space"} 9961472.0 +jvm_memory_pool_collection_max_bytes{pool="PS Old Gen"} 2.863661056E9 +# TYPE jvm_memory_pool_collection_init_bytes gauge +# HELP jvm_memory_pool_collection_init_bytes Initial after last collection bytes of a given JVM memory pool. +jvm_memory_pool_collection_init_bytes{pool="PS Eden Space"} 6.7108864E7 +jvm_memory_pool_collection_init_bytes{pool="PS Survivor Space"} 1.1010048E7 +jvm_memory_pool_collection_init_bytes{pool="PS Old Gen"} 1.79306496E8 +# TYPE jvm info +# HELP jvm VM version info +jvm_info{runtime="Java(TM) SE Runtime Environment",vendor="Oracle Corporation",version="1.8.0_212-b10"} 1.0 +# EOF \ No newline at end of file diff --git a/docs/en/seatunnel-engine/prometheus/metrics.txt b/docs/en/seatunnel-engine/prometheus/metrics.txt new file mode 100644 index 00000000000..e43dc6934a5 --- /dev/null +++ b/docs/en/seatunnel-engine/prometheus/metrics.txt @@ -0,0 +1,275 @@ +# HELP job_count All job counts of seatunnel cluster +# TYPE job_count gauge +job_count{cluster="seatunnel",type="canceled",} 0.0 +job_count{cluster="seatunnel",type="cancelling",} 0.0 +job_count{cluster="seatunnel",type="created",} 0.0 +job_count{cluster="seatunnel",type="failed",} 0.0 +job_count{cluster="seatunnel",type="failing",} 0.0 +job_count{cluster="seatunnel",type="finished",} 0.0 +job_count{cluster="seatunnel",type="reconciling",} 0.0 +job_count{cluster="seatunnel",type="restarting",} 0.0 +job_count{cluster="seatunnel",type="running",} 0.0 +job_count{cluster="seatunnel",type="scheduled",} 0.0 +job_count{cluster="seatunnel",type="suspended",} 0.0 +# HELP jvm_classes_currently_loaded The number of classes that are currently loaded in the JVM +# TYPE jvm_classes_currently_loaded gauge +jvm_classes_currently_loaded 8186.0 +# HELP jvm_classes_loaded_total The total number of classes that have been loaded since the JVM has started execution +# TYPE jvm_classes_loaded_total counter +jvm_classes_loaded_total 8186.0 +# HELP jvm_classes_unloaded_total The total number of classes that have been unloaded since the JVM has started execution +# TYPE jvm_classes_unloaded_total counter +jvm_classes_unloaded_total 0.0 +# HELP job_thread_pool_activeCount The activeCount of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_activeCount gauge +job_thread_pool_activeCount{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_completedTask_total The completedTask of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_completedTask_total counter +job_thread_pool_completedTask_total{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_corePoolSize The corePoolSize of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_corePoolSize gauge +job_thread_pool_corePoolSize{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_maximumPoolSize The maximumPoolSize of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_maximumPoolSize gauge +job_thread_pool_maximumPoolSize{cluster="seatunnel",address="[localhost]:5801",} 2.147483647E9 +# HELP job_thread_pool_poolSize The poolSize of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_poolSize gauge +job_thread_pool_poolSize{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP job_thread_pool_task_total The taskCount of seatunnel coordinator job's executor cached thread pool +# TYPE job_thread_pool_task_total counter +job_thread_pool_task_total{cluster="seatunnel",address="[localhost]:5801",} 0.0 +# HELP jvm_info VM version info +# TYPE jvm_info gauge +jvm_info{runtime="Java(TM) SE Runtime Environment",vendor="Oracle Corporation",version="1.8.0_212-b10",} 1.0 +# HELP jvm_buffer_pool_used_bytes Used bytes of a given JVM buffer pool. +# TYPE jvm_buffer_pool_used_bytes gauge +jvm_buffer_pool_used_bytes{pool="direct",} 150449.0 +jvm_buffer_pool_used_bytes{pool="mapped",} 0.0 +# HELP jvm_buffer_pool_capacity_bytes Bytes capacity of a given JVM buffer pool. +# TYPE jvm_buffer_pool_capacity_bytes gauge +jvm_buffer_pool_capacity_bytes{pool="direct",} 150447.0 +jvm_buffer_pool_capacity_bytes{pool="mapped",} 0.0 +# HELP jvm_buffer_pool_used_buffers Used buffers of a given JVM buffer pool. +# TYPE jvm_buffer_pool_used_buffers gauge +jvm_buffer_pool_used_buffers{pool="direct",} 5.0 +jvm_buffer_pool_used_buffers{pool="mapped",} 0.0 +# HELP jvm_memory_pool_allocated_bytes_total Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously. +# TYPE jvm_memory_pool_allocated_bytes_total counter +jvm_memory_pool_allocated_bytes_total{pool="Code Cache",} 8709504.0 +jvm_memory_pool_allocated_bytes_total{pool="PS Eden Space",} 4.9551024E7 +jvm_memory_pool_allocated_bytes_total{pool="PS Old Gen",} 1.1479696E7 +jvm_memory_pool_allocated_bytes_total{pool="PS Survivor Space",} 8889712.0 +jvm_memory_pool_allocated_bytes_total{pool="Compressed Class Space",} 4552568.0 +jvm_memory_pool_allocated_bytes_total{pool="Metaspace",} 3.256092E7 +# HELP jvm_gc_collection_seconds Time spent in a given JVM garbage collector in seconds. +# TYPE jvm_gc_collection_seconds summary +jvm_gc_collection_seconds_count{gc="PS Scavenge",} 8.0 +jvm_gc_collection_seconds_sum{gc="PS Scavenge",} 0.117 +jvm_gc_collection_seconds_count{gc="PS MarkSweep",} 2.0 +jvm_gc_collection_seconds_sum{gc="PS MarkSweep",} 0.071 +# HELP process_cpu_seconds_total Total user and system CPU time spent in seconds. +# TYPE process_cpu_seconds_total counter +process_cpu_seconds_total 13.922742 +# HELP process_start_time_seconds Start time of the process since unix epoch in seconds. +# TYPE process_start_time_seconds gauge +process_start_time_seconds 1.689662228826E9 +# HELP process_open_fds Number of open file descriptors. +# TYPE process_open_fds gauge +process_open_fds 113.0 +# HELP process_max_fds Maximum number of open file descriptors. +# TYPE process_max_fds gauge +process_max_fds 10240.0 +# HELP jvm_memory_objects_pending_finalization The number of objects waiting in the finalizer queue. +# TYPE jvm_memory_objects_pending_finalization gauge +jvm_memory_objects_pending_finalization 0.0 +# HELP jvm_memory_bytes_used Used bytes of a given JVM memory area. +# TYPE jvm_memory_bytes_used gauge +jvm_memory_bytes_used{area="heap",} 2.10046672E8 +jvm_memory_bytes_used{area="nonheap",} 5.7037864E7 +# HELP jvm_memory_bytes_committed Committed (bytes) of a given JVM memory area. +# TYPE jvm_memory_bytes_committed gauge +jvm_memory_bytes_committed{area="heap",} 6.01358336E8 +jvm_memory_bytes_committed{area="nonheap",} 6.221824E7 +# HELP jvm_memory_bytes_max Max (bytes) of a given JVM memory area. +# TYPE jvm_memory_bytes_max gauge +jvm_memory_bytes_max{area="heap",} 3.817865216E9 +jvm_memory_bytes_max{area="nonheap",} -1.0 +# HELP jvm_memory_bytes_init Initial bytes of a given JVM memory area. +# TYPE jvm_memory_bytes_init gauge +jvm_memory_bytes_init{area="heap",} 2.68435456E8 +jvm_memory_bytes_init{area="nonheap",} 2555904.0 +# HELP jvm_memory_pool_bytes_used Used bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_used gauge +jvm_memory_pool_bytes_used{pool="Code Cache",} 1.0777728E7 +jvm_memory_pool_bytes_used{pool="Metaspace",} 4.080908E7 +jvm_memory_pool_bytes_used{pool="Compressed Class Space",} 5451056.0 +jvm_memory_pool_bytes_used{pool="PS Eden Space",} 1.98927776E8 +jvm_memory_pool_bytes_used{pool="PS Survivor Space",} 0.0 +jvm_memory_pool_bytes_used{pool="PS Old Gen",} 1.1118896E7 +# HELP jvm_memory_pool_bytes_committed Committed bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_committed gauge +jvm_memory_pool_bytes_committed{pool="Code Cache",} 1.1010048E7 +jvm_memory_pool_bytes_committed{pool="Metaspace",} 4.4867584E7 +jvm_memory_pool_bytes_committed{pool="Compressed Class Space",} 6340608.0 +jvm_memory_pool_bytes_committed{pool="PS Eden Space",} 4.9283072E8 +jvm_memory_pool_bytes_committed{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_bytes_committed{pool="PS Old Gen",} 1.02760448E8 +# HELP jvm_memory_pool_bytes_max Max bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_max gauge +jvm_memory_pool_bytes_max{pool="Code Cache",} 2.5165824E8 +jvm_memory_pool_bytes_max{pool="Metaspace",} -1.0 +jvm_memory_pool_bytes_max{pool="Compressed Class Space",} 1.073741824E9 +jvm_memory_pool_bytes_max{pool="PS Eden Space",} 1.414004736E9 +jvm_memory_pool_bytes_max{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_bytes_max{pool="PS Old Gen",} 2.863661056E9 +# HELP jvm_memory_pool_bytes_init Initial bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_bytes_init gauge +jvm_memory_pool_bytes_init{pool="Code Cache",} 2555904.0 +jvm_memory_pool_bytes_init{pool="Metaspace",} 0.0 +jvm_memory_pool_bytes_init{pool="Compressed Class Space",} 0.0 +jvm_memory_pool_bytes_init{pool="PS Eden Space",} 6.7108864E7 +jvm_memory_pool_bytes_init{pool="PS Survivor Space",} 1.1010048E7 +jvm_memory_pool_bytes_init{pool="PS Old Gen",} 1.79306496E8 +# HELP jvm_memory_pool_collection_used_bytes Used bytes after last collection of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_used_bytes gauge +jvm_memory_pool_collection_used_bytes{pool="PS Eden Space",} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Survivor Space",} 0.0 +jvm_memory_pool_collection_used_bytes{pool="PS Old Gen",} 1.1118896E7 +# HELP jvm_memory_pool_collection_committed_bytes Committed after last collection bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_committed_bytes gauge +jvm_memory_pool_collection_committed_bytes{pool="PS Eden Space",} 4.9283072E8 +jvm_memory_pool_collection_committed_bytes{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_collection_committed_bytes{pool="PS Old Gen",} 1.02760448E8 +# HELP jvm_memory_pool_collection_max_bytes Max bytes after last collection of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_max_bytes gauge +jvm_memory_pool_collection_max_bytes{pool="PS Eden Space",} 1.414004736E9 +jvm_memory_pool_collection_max_bytes{pool="PS Survivor Space",} 5767168.0 +jvm_memory_pool_collection_max_bytes{pool="PS Old Gen",} 2.863661056E9 +# HELP jvm_memory_pool_collection_init_bytes Initial after last collection bytes of a given JVM memory pool. +# TYPE jvm_memory_pool_collection_init_bytes gauge +jvm_memory_pool_collection_init_bytes{pool="PS Eden Space",} 6.7108864E7 +jvm_memory_pool_collection_init_bytes{pool="PS Survivor Space",} 1.1010048E7 +jvm_memory_pool_collection_init_bytes{pool="PS Old Gen",} 1.79306496E8 +# HELP node_state Whether is up of seatunnel node +# TYPE node_state gauge +node_state{cluster="seatunnel",address="[localhost]:5801",} 1.0 +# HELP node_count Cluster node total count +# TYPE node_count gauge +node_count{cluster="seatunnel",} 1.0 +# HELP hazelcast_executor_executedCount The hazelcast executor executedCount of seatunnel cluster node +# TYPE hazelcast_executor_executedCount gauge +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="io",} 128.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 402.0 +hazelcast_executor_executedCount{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_isShutdown The hazelcast executor isShutdown of seatunnel cluster node +# TYPE hazelcast_executor_isShutdown gauge +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_isShutdown{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_isTerminated The hazelcast executor isTerminated of seatunnel cluster node +# TYPE hazelcast_executor_isTerminated gauge +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_isTerminated{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_maxPoolSize The hazelcast executor maxPoolSize of seatunnel cluster node +# TYPE hazelcast_executor_maxPoolSize gauge +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="async",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="client",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 160.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="io",} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 8.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 16.0 +hazelcast_executor_maxPoolSize{cluster="seatunnel",address="[localhost]:5801",type="system",} 8.0 +# HELP hazelcast_executor_poolSize The hazelcast executor poolSize of seatunnel cluster node +# TYPE hazelcast_executor_poolSize gauge +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_poolSize{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_executor_queueRemainingCapacity The hazelcast executor queueRemainingCapacity of seatunnel cluster +# TYPE hazelcast_executor_queueRemainingCapacity gauge +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="async",} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="client",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="io",} 2.147483647E9 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 100000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 800000.0 +hazelcast_executor_queueRemainingCapacity{cluster="seatunnel",address="[localhost]:5801",type="system",} 2.147483647E9 +# HELP hazelcast_executor_queueSize The hazelcast executor queueSize of seatunnel cluster node +# TYPE hazelcast_executor_queueSize gauge +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="async",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="client",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="clientBlocking",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="clientQuery",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="io",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="offloadable",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="scheduled",} 0.0 +hazelcast_executor_queueSize{cluster="seatunnel",address="[localhost]:5801",type="system",} 0.0 +# HELP hazelcast_partition_partitionCount The partitionCount of seatunnel cluster node +# TYPE hazelcast_partition_partitionCount gauge +hazelcast_partition_partitionCount{cluster="seatunnel",address="[localhost]:5801",} 271.0 +# HELP hazelcast_partition_activePartition The activePartition of seatunnel cluster node +# TYPE hazelcast_partition_activePartition gauge +hazelcast_partition_activePartition{cluster="seatunnel",address="[localhost]:5801",} 271.0 +# HELP hazelcast_partition_isClusterSafe Whether is cluster safe of partition +# TYPE hazelcast_partition_isClusterSafe gauge +hazelcast_partition_isClusterSafe{cluster="seatunnel",address="[localhost]:5801",} 1.0 +# HELP hazelcast_partition_isLocalMemberSafe Whether is local member safe of partition +# TYPE hazelcast_partition_isLocalMemberSafe gauge +hazelcast_partition_isLocalMemberSafe{cluster="seatunnel",address="[localhost]:5801",} 1.0 +# HELP jvm_threads_current Current thread count of a JVM +# TYPE jvm_threads_current gauge +jvm_threads_current 114.0 +# HELP jvm_threads_daemon Daemon thread count of a JVM +# TYPE jvm_threads_daemon gauge +jvm_threads_daemon 5.0 +# HELP jvm_threads_peak Peak thread count of a JVM +# TYPE jvm_threads_peak gauge +jvm_threads_peak 116.0 +# HELP jvm_threads_started_total Started thread count of a JVM +# TYPE jvm_threads_started_total counter +jvm_threads_started_total 119.0 +# HELP jvm_threads_deadlocked Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers +# TYPE jvm_threads_deadlocked gauge +jvm_threads_deadlocked 0.0 +# HELP jvm_threads_deadlocked_monitor Cycles of JVM-threads that are in deadlock waiting to acquire object monitors +# TYPE jvm_threads_deadlocked_monitor gauge +jvm_threads_deadlocked_monitor 0.0 +# HELP jvm_threads_state Current count of threads by state +# TYPE jvm_threads_state gauge +jvm_threads_state{state="NEW",} 0.0 +jvm_threads_state{state="TERMINATED",} 0.0 +jvm_threads_state{state="RUNNABLE",} 10.0 +jvm_threads_state{state="BLOCKED",} 0.0 +jvm_threads_state{state="WAITING",} 75.0 +jvm_threads_state{state="TIMED_WAITING",} 29.0 +jvm_threads_state{state="UNKNOWN",} 0.0 +# HELP jvm_memory_pool_allocated_bytes_created Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously. +# TYPE jvm_memory_pool_allocated_bytes_created gauge +jvm_memory_pool_allocated_bytes_created{pool="Code Cache",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="PS Eden Space",} 1.689662236757E9 +jvm_memory_pool_allocated_bytes_created{pool="PS Old Gen",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="PS Survivor Space",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="Compressed Class Space",} 1.689662236806E9 +jvm_memory_pool_allocated_bytes_created{pool="Metaspace",} 1.689662236806E9 \ No newline at end of file diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index c880c26f8bb..989e8ac0b80 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -15,71 +15,132 @@ The following is an example declarative configuration. seatunnel: engine: telemetry: - http-port: 9090 # The port exposed by the telemetry server, default is 9090. - load-default-exports: true # Whether to load default jvm exports, default is true. + metric: + enabled: true # Whether open metrics export ``` ## Metrics +The [metric text of prometheus](./prometheus/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/metrics`. + +The [metric text of openMetrics](./openmetrics/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/openmetrics`. + Available metrics include the following categories. +Note: All metrics both have the same labelName `cluster`, that's value is the config of `hazelcast.cluster-name`. + +### Node Metrics + +| MetricName | Type | Labels | DESCRIPTION | +|-------------------------------------------|-------|------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------| +| cluster_info | Gauge | **hazelcastVersion**, hazelcastVersion
,*master**, seatunnel master address | Cluster info | +| cluster_time | Gauge | **hazelcastVersion**, hazelcastVersion
,*version**, seatunnel version | Cluster time | +| node_count | Gauge | - | Cluster node total count | +| node_state | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | Whether is up of seatunnel node | +| hazelcast_executor_executedCount | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor executedCount of seatunnel cluster node | +| hazelcast_executor_isShutdown | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor isShutdown of seatunnel cluster node | +| hazelcast_executor_isTerminated | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor isTerminated of seatunnel cluster node | +| hazelcast_executor_maxPoolSize | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor maxPoolSize of seatunnel cluster node | +| hazelcast_executor_poolSize | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor poolSize of seatunnel cluster node | +| hazelcast_executor_queueRemainingCapacity | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor queueRemainingCapacity of seatunnel cluster node | +| hazelcast_executor_queueSize | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor queueSize of seatunnel cluster node | +| hazelcast_partition_partitionCount | Gauge | - | The partitionCount of seatunnel cluster node | +| hazelcast_partition_activePartition | Gauge | - | The activePartition of seatunnel cluster node | +| hazelcast_partition_isClusterSafe | Gauge | - | Weather is cluster safe of partition | +| hazelcast_partition_isLocalMemberSafe | Gauge | - | Weather is local member safe of partition | + ### Thread Pool Status -| MetricName | Type | Labels | -|-------------------------------------|---------|----------------------------------------------------------------------| -| job_thread_pool_activeCount | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_corePoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_maximumPoolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_poolSize | Gauge | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_completedTask_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | -| job_thread_pool_task_total | Counter | **address**, server instance address,for example: "[localhost]:5801" | +| MetricName | Type | Labels | DESCRIPTION | +|-------------------------------------|---------|--------------------------------------------------------------------|--------------------------------------------------------------------------------| +| job_thread_pool_activeCount | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The activeCount of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_corePoolSize | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The corePoolSize of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_maximumPoolSize | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The maximumPoolSize of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_poolSize | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | The poolSize of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_completedTask_total | Counter | **address**, server instance address,for example: "127.0.0.1:5801" | The completedTask of seatunnel coordinator job's executor cached thread pool | +| job_thread_pool_task_total | Counter | **address**, server instance address,for example: "127.0.0.1:5801" | The taskCount of seatunnel coordinator job's executor cached thread pool | ### Job info detail -| MetricName | Type | Labels | -|------------|-------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| job_count | Gauge | **type**, the type of job, including: "canceled" "cancelling" "created" "failed" "failing" "finished" "reconciling" "restarting" "running" "scheduled" "suspended" | +| MetricName | Type | Labels | DESCRIPTION | +|------------|-------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------| +| job_count | Gauge | **type**, the type of job, including: "canceled" "cancelling" "created" "failed" "failing" "finished" "reconciling" "restarting" "running" "scheduled" "suspended" | All job counts of seatunnel cluster | ### JVM Metrics -| MetricName | Type | Labels | -|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------| -| jvm_threads_current | Gauge | - | -| jvm_threads_daemon | Gauge | - | -| jvm_threads_daemon | Gauge | - | -| jvm_threads_peak | Gauge | - | -| jvm_threads_started_total | Counter | - | -| jvm_threads_deadlocked | Gauge | - | -| jvm_threads_deadlocked_monitor | Gauge | - | -| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | -| jvm_classes_currently_loaded | Gauge | - | -| jvm_classes_loaded_total | Counter | - | -| jvm_classes_unloaded_total | Counter | - | -| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | -| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | -| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | -| process_cpu_seconds_total | Counter | - | -| process_start_time_seconds | Gauge | - | -| process_open_fds | Gauge | - | -| process_max_fds | Gauge | - | -| jvm_memory_objects_pending_finalization | Gauge | - | -| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | -| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | -| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | -| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | -| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | -| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | -| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | -| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | -| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | - -### 实现方案 +| MetricName | Type | Labels | DESCRIPTION | +|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------| +| jvm_threads_current | Gauge | - | Current thread count of a JVM | +| jvm_threads_daemon | Gauge | - | Daemon thread count of a JVM | +| jvm_threads_peak | Gauge | - | Peak thread count of a JVM | +| jvm_threads_started_total | Counter | - | Started thread count of a JVM | +| jvm_threads_deadlocked | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers | +| jvm_threads_deadlocked_monitor | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors | +| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | Current count of threads by state | +| jvm_classes_currently_loaded | Gauge | - | The number of classes that are currently loaded in the JVM | +| jvm_classes_loaded_total | Counter | - | The total number of classes that have been loaded since the JVM has started execution | +| jvm_classes_unloaded_total | Counter | - | The total number of classes that have been unloaded since the JVM has started execution | +| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | VM version info | +| process_cpu_seconds_total | Counter | - | Total user and system CPU time spent in seconds | +| process_start_time_seconds | Gauge | - | Start time of the process since unix epoch in seconds | +| process_open_fds | Gauge | - | Number of open file descriptors | +| process_max_fds | Gauge | - | Maximum number of open file descriptors | +| jvm_memory_objects_pending_finalization | Gauge | - | The number of objects waiting in the finalizer queue | +| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | Used bytes of a given JVM memory area | +| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | Committed (bytes) of a given JVM memory area | +| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | Max (bytes) of a given JVM memory area | +| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | Initial bytes of a given JVM memory area | +| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Used bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Committed bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Max bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Initial bytes of a given JVM memory pool | +| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Used bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Committed after last collection bytes of a given JVM memory pool | +| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Max bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Initial after last collection bytes of a given JVM memory pool | +| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | Used bytes of a given JVM buffer pool | +| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | Bytes capacity of a given JVM buffer pool | +| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | Used buffers of a given JVM buffer pool | + +### Cluster Monitoring By Prometheus & Grafana + +#### Install Prometheus + +For a guide on how to set up Prometheus server go to the [Installation](https://prometheus.io/docs/prometheus/latest/installation) + +#### Configuration Prometheus + +Add seatunnel instance metric exports into `/etc/prometheus/prometheus.yaml`. For example: + +```yaml +global: + # How frequently to scrape targets from this job. + scrape_interval: 15s + +scrape_configs: + # The job name assigned to scraped metrics by default. + - job_name: 'seatunnel' + scrape_interval: 5s + # List of labeled statically configured targets for this job. + static_configs: + # The targets specified by the static config. + - targets: ['localhost:5701'] + # Labels assigned to all metrics scraped from the targets. + # labels: [:] +``` + +#### Install Grafana + +For a guide on how to set up Grafana server go to the [Installation](https://grafana.com/docs/grafana/latest/setup-grafana/installation) + +#### Monitoring Dashboard + +- Add Prometheus DataSource on Grafana. +- Import `Seatunnel Cluster` monitoring dashboard by [Dashboard JSON](./grafana/grafanaDashboard.json) into Grafana. + +The [effect image](../images/cluster_monitoring.png) of the dashboard diff --git a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java index e5d11fc3b5c..ec3872e449f 100644 --- a/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java +++ b/seatunnel-core/seatunnel-starter/src/main/java/org/apache/seatunnel/core/starter/seatunnel/command/ServerExecuteCommand.java @@ -25,8 +25,6 @@ import org.apache.commons.lang3.StringUtils; -import java.io.IOException; - /** This command is used to execute the SeaTunnel engine job by SeaTunnel API. */ public class ServerExecuteCommand implements Command { @@ -37,7 +35,7 @@ public ServerExecuteCommand(ServerCommandArgs serverCommandArgs) { } @Override - public void execute() throws IOException { + public void execute() { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); if (StringUtils.isNotEmpty(serverCommandArgs.getClusterName())) { seaTunnelConfig.getHazelcastConfig().setClusterName(serverCommandArgs.getClusterName()); diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index dca007e2f27..61b5cec0442 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -275,13 +275,13 @@ The text of each license is the standard Apache 2.0 license. (Apache-2.0) listenablefuture (com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava https://mvnrepository.com/artifact/com.google.guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava) (Apache-2.0) accessors-smart (com.google.guava:accessors-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/accessors-smart) (Apache-2.0) json-smart (net.minidev:json-smart:2.4.7 - https://mvnrepository.com/artifact/net.minidev/json-smart) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient (io.prometheus:simpleclient:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Common (io.prometheus:simpleclient_common:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Hotspot (io.prometheus:simpleclient_hotspot:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Httpserver (io.prometheus:simpleclient_httpserver:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - Common (io.prometheus:simpleclient_tracer_common:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry (io.prometheus:simpleclient_tracer_otel:0.16.0 - https://www.apache.org/hive-storage-api/) - (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry Agent (io.prometheus:simpleclient_tracer_otel_agent:0.16.0 - https://www.apache.org/hive-storage-api/) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient (io.prometheus:simpleclient:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Common (io.prometheus:simpleclient_common:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_common/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Hotspot (io.prometheus:simpleclient_hotspot:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_hotspot/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Simpleclient Httpserver (io.prometheus:simpleclient_httpserver:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_httpserver/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - Common (io.prometheus:simpleclient_tracer_common:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_tracer_common/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry (io.prometheus:simpleclient_tracer_otel:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_tracer_otel/0.16.0) + (The Apache Software License, Version 2.0) Prometheus Java Span Context Supplier - OpenTelemetry Agent (io.prometheus:simpleclient_tracer_otel_agent:0.16.0 - https://mvnrepository.com/artifact/io.prometheus/simpleclient_tracer_otel_agent/0.16.0) ======================================================================== MOZILLA PUBLIC LICENSE License diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java index f8891da7401..8c087b27c3d 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/JobExecutionIT.java @@ -48,7 +48,7 @@ public class JobExecutionIT { private static HazelcastInstanceImpl hazelcastInstance; @BeforeAll - public static void beforeClass() throws Exception { + public static void beforeClass() { hazelcastInstance = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName("JobExecutionIT")); diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java index c3b2129c5c3..030d471fa5c 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.JobConfig; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; import org.apache.seatunnel.engine.core.job.JobStatus; import org.apache.seatunnel.engine.server.SeaTunnelServerStarter; @@ -54,18 +53,16 @@ public class TelemetryApiIT { private static HazelcastInstanceImpl hazelcastInstance; - private static TelemetryMetricConfig metricConfig; + private static String testClusterName; @BeforeAll static void beforeClass() throws Exception { - String testClusterName = TestUtils.getClusterName("TelemetryApiIT"); + testClusterName = TestUtils.getClusterName("TelemetryApiIT"); SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); seaTunnelConfig.getHazelcastConfig().setClusterName(testClusterName); - // get TelemetryMetricConfig - metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); hazelcastInstance = SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig); // createTelemetryInstance - SeaTunnelServerStarter.createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + SeaTunnelServerStarter.initTelemetryInstance(hazelcastInstance.node); Common.setDeployMode(DeployMode.CLIENT); String filePath = TestUtils.getResource("stream_fakesource_to_file.conf"); JobConfig jobConfig = new JobConfig(); @@ -89,7 +86,14 @@ static void beforeClass() throws Exception { @Test public void testGetMetrics() throws InterruptedException { - given().get(HOST + metricConfig.getHttpPort() + "/metrics") + given().get( + HOST + + hazelcastInstance + .getCluster() + .getLocalMember() + .getAddress() + .getPort() + + "/hazelcast/rest/instance/metrics") .then() .statusCode(200) // Use regular expressions to verify whether the response body is the indicator data @@ -106,30 +110,419 @@ public void testGetMetrics() throws InterruptedException { .body(containsString("jvm_classes")) .body(containsString("jvm_buffer_pool")) .body(containsString("process_start")) + // + .body(containsString("cluster_info{cluster=\"" + testClusterName)) + // + .body(containsString("cluster_time{cluster=\"" + testClusterName)) // Job thread pool metrics - .body(containsString("job_thread_pool_activeCount{address=\"[localhost]:5801\",}")) .body( containsString( - "job_thread_pool_completedTask_total{address=\"[localhost]:5801\",}")) - .body(containsString("job_thread_pool_corePoolSize{address=\"[localhost]:5801\",}")) + "job_thread_pool_activeCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) .body( containsString( - "job_thread_pool_maximumPoolSize{address=\"[localhost]:5801\",} 2.147483647E9")) - .body(containsString("job_thread_pool_poolSize{address=\"[localhost]:5801\",}")) - .body(containsString("job_thread_pool_task_total{address=\"[localhost]:5801\",}")) + "job_thread_pool_completedTask_total{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + .body( + containsString( + "job_thread_pool_corePoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + .body( + containsString( + "job_thread_pool_maximumPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",} 2.147483647E9")) + .body( + containsString( + "job_thread_pool_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + .body( + containsString( + "job_thread_pool_task_total{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) // Job count metrics - .body(containsString("job_count{type=\"canceled\",} 0.0")) - .body(containsString("job_count{type=\"cancelling\",} 0.0")) - .body(containsString("job_count{type=\"created\",} 0.0")) - .body(containsString("job_count{type=\"failed\",} 0.0")) - .body(containsString("job_count{type=\"failing\",} 0.0")) - .body(containsString("job_count{type=\"finished\",} 0.0")) - .body(containsString("job_count{type=\"reconciling\",} 0.0")) - .body(containsString("job_count{type=\"restarting\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"canceled\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"cancelling\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"created\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"failed\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"failing\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"finished\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"reconciling\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"restarting\",} 0.0")) // Running job count is 1 - .body(containsString("job_count{type=\"running\",} 1.0")) - .body(containsString("job_count{type=\"scheduled\",} 0.0")) - .body(containsString("job_count{type=\"suspended\",} 0.0")); + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"running\",} 1.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"scheduled\",} 0.0")) + .body( + containsString( + "job_count{cluster=\"" + + testClusterName + + "\",type=\"suspended\",} 0.0")) + // Node + .body( + containsString( + "node_state{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_executor_executedCount + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_executedCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + // hazelcast_executor_isShutdown + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_isShutdown{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + // hazelcast_executor_isTerminated + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_isTerminated{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_maxPoolSize + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_maxPoolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_poolSize + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_poolSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_queueRemainingCapacity + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_queueRemainingCapacity{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + // hazelcast_executor_queueSize + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + .body( + containsString( + "hazelcast_executor_queueSize{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + // hazelcast_partition_partitionCount + .body( + containsString( + "hazelcast_partition_partitionCount{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_partition_activePartition + .body( + containsString( + "hazelcast_partition_activePartition{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_partition_isClusterSafe + .body( + containsString( + "hazelcast_partition_isClusterSafe{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")) + // hazelcast_partition_isLocalMemberSafe + .body( + containsString( + "hazelcast_partition_isLocalMemberSafe{cluster=\"" + + testClusterName + + "\",address=\"127.0.0.1:5801\",}")); } @AfterAll diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml index e1048ab83f9..234493d29a6 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml @@ -42,4 +42,3 @@ hazelcast: hazelcast.slow.operation.detector.stacktrace.logging.enabled: true hazelcast.logging.type: log4j2 hazelcast.operation.generic.thread.count: 200 -# hazelcast.jet.enabled: true diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java index bf0d17f65ee..e2618528fa6 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/YamlSeaTunnelDomConfigProcessor.java @@ -234,15 +234,8 @@ private TelemetryMetricConfig parseTelemetryMetricConfig(Node metricNode) { TelemetryMetricConfig metricConfig = new TelemetryMetricConfig(); for (Node node : childElements(metricNode)) { String name = cleanNodeName(node); - if (ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key().equals(name)) { - metricConfig.setHttpPort( - getIntegerValue( - ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.key(), - getTextContent(node))); - } else if (ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS - .key() - .equals(name)) { - metricConfig.setLoadDefaultExports(getBooleanValue(getTextContent(node))); + if (ServerConfigOptions.TELEMETRY_METRIC_ENABLED.key().equals(name)) { + metricConfig.setEnabled(getBooleanValue(getTextContent(node))); } else { LOGGER.warning("Unrecognized element: " + name); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java index 5a3e9e55024..e72f66f5c5b 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/ServerConfigOptions.java @@ -146,17 +146,11 @@ public class ServerConfigOptions { .noDefaultValue() .withDescription("The checkpoint storage instance configuration."); - public static final Option TELEMETRY_METRIC_HTTP_PORT = - Options.key("http-port") - .intType() - .defaultValue(9090) - .withDescription("The telemetry metric server's port."); - - public static final Option TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS = - Options.key("load-default-exports") + public static final Option TELEMETRY_METRIC_ENABLED = + Options.key("enabled") .booleanType() .defaultValue(true) - .withDescription("Whether to load default jvm exports."); + .withDescription("Whether open metrics export."); public static final Option TELEMETRY_METRIC = Options.key("metric") diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java index c4d10cdd0b5..a211514e353 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java +++ b/seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/config/server/TelemetryMetricConfig.java @@ -21,19 +21,8 @@ import java.io.Serializable; -import static com.google.common.base.Preconditions.checkArgument; - @Data public class TelemetryMetricConfig implements Serializable { - private int httpPort = ServerConfigOptions.TELEMETRY_METRIC_HTTP_PORT.defaultValue(); - private boolean loadDefaultExports = - ServerConfigOptions.TELEMETRY_METRIC_LOAD_DEFAULT_EXPORTS.defaultValue(); - - public void setHttpPort(int httpPort) { - checkArgument( - httpPort >= 0, - "The number of http's port failed checkpoints must be a natural number."); - this.httpPort = httpPort; - } + private boolean enabled = ServerConfigOptions.TELEMETRY_METRIC_ENABLED.defaultValue(); } diff --git a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml index 9ceb2ed68a3..3fbd1896c67 100644 --- a/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml +++ b/seatunnel-engine/seatunnel-engine-common/src/main/resources/seatunnel.yaml @@ -36,5 +36,4 @@ seatunnel: fs.defaultFS: file:///tmp/ telemetry: metric: - http-port: 9090 - load-default-exports: true + enabled: true diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java index 37e00cffab2..6690ae02f5d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/NodeExtension.java @@ -28,6 +28,8 @@ import com.hazelcast.instance.impl.Node; import com.hazelcast.internal.ascii.TextCommandService; import com.hazelcast.internal.ascii.TextCommandServiceImpl; +import io.prometheus.client.CollectorRegistry; +import lombok.Getter; import lombok.NonNull; import java.util.Map; @@ -37,10 +39,12 @@ public class NodeExtension extends DefaultNodeExtension { private final NodeExtensionCommon extCommon; + @Getter private CollectorRegistry collectorRegistry; public NodeExtension(@NonNull Node node, @NonNull SeaTunnelConfig seaTunnelConfig) { super(node); extCommon = new NodeExtensionCommon(node, new SeaTunnelServer(seaTunnelConfig)); + collectorRegistry = CollectorRegistry.defaultRegistry; } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java index 51e18cfea5f..10463cd63bd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServerStarter.java @@ -19,7 +19,7 @@ import org.apache.seatunnel.engine.common.config.ConfigProvider; import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstanceFactory; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstanceInitializer; import com.hazelcast.instance.impl.HazelcastInstanceFactory; import com.hazelcast.instance.impl.HazelcastInstanceImpl; @@ -27,25 +27,23 @@ import com.hazelcast.instance.impl.Node; import lombok.NonNull; -import java.io.IOException; - public class SeaTunnelServerStarter { - public static void main(String[] args) throws IOException { + public static void main(String[] args) { createHazelcastInstance(); } - public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) - throws IOException { + public static HazelcastInstanceImpl createHazelcastInstance(String clusterName) { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); seaTunnelConfig.getHazelcastConfig().setClusterName(clusterName); HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); - createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + initTelemetryInstance(hazelcastInstance.node); return hazelcastInstance; } public static HazelcastInstanceImpl createHazelcastInstance( @NonNull SeaTunnelConfig seaTunnelConfig) { + checkTelemetryConfig(seaTunnelConfig); return ((HazelcastInstanceProxy) HazelcastInstanceFactory.newHazelcastInstance( seaTunnelConfig.getHazelcastConfig(), @@ -55,15 +53,24 @@ public static HazelcastInstanceImpl createHazelcastInstance( .getOriginal(); } - public static HazelcastInstanceImpl createHazelcastInstance() throws IOException { + public static HazelcastInstanceImpl createHazelcastInstance() { SeaTunnelConfig seaTunnelConfig = ConfigProvider.locateAndGetSeaTunnelConfig(); HazelcastInstanceImpl hazelcastInstance = createHazelcastInstance(seaTunnelConfig); - createTelemetryInstance(hazelcastInstance.node, seaTunnelConfig); + initTelemetryInstance(hazelcastInstance.node); return hazelcastInstance; } - public static void createTelemetryInstance( - @NonNull Node node, final SeaTunnelConfig seaTunnelConfig) throws IOException { - ExportsInstanceFactory.newExportsInstance(node, seaTunnelConfig); + public static void initTelemetryInstance(@NonNull Node node) { + ExportsInstanceInitializer.init(node); + } + + private static void checkTelemetryConfig(SeaTunnelConfig seaTunnelConfig) { + // "hazelcast.jmx" need to set "true", for hazelcast metrics + if (seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric().isEnabled()) { + seaTunnelConfig + .getHazelcastConfig() + .getProperties() + .setProperty("hazelcast.jmx", "true"); + } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java index 7776d592b8f..94ba5e1a2fe 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestConstant.java @@ -25,4 +25,7 @@ public class RestConstant { public static final String SYSTEM_MONITORING_INFORMATION = "/hazelcast/rest/maps/system-monitoring-information"; + + public static final String TELEMETRY_METRICS_URL = "/hazelcast/rest/instance/metrics"; + public static final String TELEMETRY_OPEN_METRICS_URL = "/hazelcast/rest/instance/openmetrics"; } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java index 6c71ac2feb3..5d22772bb8f 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/rest/RestHttpGetCommandProcessor.java @@ -46,7 +46,10 @@ import com.hazelcast.jet.impl.execution.init.CustomClassLoadedObject; import com.hazelcast.map.IMap; import com.hazelcast.spi.impl.NodeEngine; +import io.prometheus.client.exporter.common.TextFormat; +import java.io.IOException; +import java.io.StringWriter; import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; @@ -59,6 +62,8 @@ import static org.apache.seatunnel.engine.server.rest.RestConstant.RUNNING_JOBS_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.RUNNING_JOB_URL; import static org.apache.seatunnel.engine.server.rest.RestConstant.SYSTEM_MONITORING_INFORMATION; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TELEMETRY_METRICS_URL; +import static org.apache.seatunnel.engine.server.rest.RestConstant.TELEMETRY_OPEN_METRICS_URL; public class RestHttpGetCommandProcessor extends HttpCommandProcessor { @@ -93,6 +98,10 @@ public void handle(HttpGetCommand httpGetCommand) { handleJobInfoById(httpGetCommand, uri); } else if (uri.startsWith(SYSTEM_MONITORING_INFORMATION)) { getSystemMonitoringInformation(httpGetCommand); + } else if (uri.equals(TELEMETRY_METRICS_URL)) { + handleMetrics(httpGetCommand, TextFormat.CONTENT_TYPE_004); + } else if (uri.equals(TELEMETRY_OPEN_METRICS_URL)) { + handleMetrics(httpGetCommand, TextFormat.CONTENT_TYPE_OPENMETRICS_100); } else { original.handle(httpGetCommand); } @@ -209,6 +218,31 @@ private Map getJobMetrics(String jobMetrics) { return metricsMap; } + private void handleMetrics(HttpGetCommand httpGetCommand, String contentType) { + StringWriter stringWriter = new StringWriter(); + org.apache.seatunnel.engine.server.NodeExtension nodeExtension = + (org.apache.seatunnel.engine.server.NodeExtension) + textCommandService.getNode().getNodeExtension(); + try { + TextFormat.writeFormat( + contentType, + stringWriter, + nodeExtension.getCollectorRegistry().metricFamilySamples()); + this.prepareResponse(httpGetCommand, stringWriter.toString()); + } catch (IOException e) { + httpGetCommand.send400(); + } finally { + try { + if (stringWriter != null) { + stringWriter.close(); + } + } catch (IOException e) { + logger.warning("An error occurred while handling request " + httpGetCommand, e); + prepareResponse(SC_500, httpGetCommand, exceptionResponse(e)); + } + } + } + private SeaTunnelServer getSeaTunnelServer() { Map extensionServices = this.textCommandService.getNode().getNodeExtension().createExtensionServices(); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java index 6547a2708b4..99a1f15e9a1 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -20,13 +20,24 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.SeaTunnelServer; +import com.google.common.collect.Lists; import com.hazelcast.cluster.impl.MemberImpl; import com.hazelcast.instance.impl.Node; +import com.hazelcast.internal.cluster.ClusterService; +import com.hazelcast.internal.jmx.ManagementService; import com.hazelcast.logging.ILogger; import io.prometheus.client.Collector; +import io.prometheus.client.GaugeMetricFamily; + +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; public abstract class AbstractCollector extends Collector { + protected static String CLUSTER = "cluster"; + protected static String ADDRESS = "address"; + protected ExportsInstance exportsInstance; public AbstractCollector(final ExportsInstance exportsInstance) { @@ -56,4 +67,55 @@ protected SeaTunnelServer getServer() { protected CoordinatorService getCoordinatorService() { return getServer().getCoordinatorService(); } + + protected ManagementService getManagementService() { + return getNode().hazelcastInstance.getManagementService(); + } + + protected ClusterService getClusterService() { + return getNode().getClusterService(); + } + + protected String localAddress() { + return getLocalMember().getInetAddress().getHostAddress() + + ":" + + getLocalMember().getPort(); + } + + protected String masterAddress() throws UnknownHostException { + return getClusterService().getMasterAddress().getInetAddress().getHostAddress() + + ":" + + getClusterService().getMasterAddress().getPort(); + } + + protected String getClusterName() { + return getNode().getConfig().getClusterName(); + } + + protected List labelValues(String... values) { + List labelValues = new ArrayList<>(); + labelValues.add(getClusterName()); + if (values != null) { + labelValues.addAll(Lists.newArrayList(values)); + } + return labelValues; + } + + protected List clusterLabelNames(String... labels) { + List labelNames = new ArrayList<>(); + labelNames.add(CLUSTER); + if (labels != null) { + labelNames.addAll(Lists.newArrayList(labels)); + } + return labelNames; + } + + protected void longMetric( + GaugeMetricFamily metricFamily, long count, List labelValues) { + metricFamily.addMetric(labelValues, count); + } + + protected void intMetric(GaugeMetricFamily metricFamily, int count, List labelValues) { + metricFamily.addMetric(labelValues, count); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java index 5f636dd6c45..66bcdb8b64b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java @@ -17,43 +17,38 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; -import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; -import org.apache.seatunnel.engine.common.config.server.TelemetryMetricConfig; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ClusterMetricExports; import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.NodeMetricExports; import com.hazelcast.instance.impl.Node; import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.exporter.HTTPServer; import io.prometheus.client.hotspot.DefaultExports; - -import java.io.IOException; +import lombok.Getter; public class ExportsInstance { - private Node node; - private TelemetryMetricConfig metricConfig; + @Getter private Node node; - public ExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) throws IOException { + public ExportsInstance(Node node) { this.node = node; - this.metricConfig = seaTunnelConfig.getEngineConfig().getTelemetryConfig().getMetric(); - start(); + init(); } - private void start() throws IOException { - if (metricConfig.isLoadDefaultExports()) { - DefaultExports.initialize(); - } - HTTPServer httpServer = - new HTTPServer.Builder().withPort(metricConfig.getHttpPort()).build(); + private void init() { + // initialize jvm collector + DefaultExports.initialize(); + + // register collectors CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; // Job info detail new JobMetricExports(this).register(collectorRegistry); // Thread pool status new JobThreadPoolStatusExports(this).register(collectorRegistry); - } - - public Node getNode() { - return node; + // Node metrics + new NodeMetricExports(this).register(collectorRegistry); + // Cluster metrics + new ClusterMetricExports(this).register(collectorRegistry); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java similarity index 71% rename from seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java rename to seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java index 29a85469cec..ccc3f206bdd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceFactory.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java @@ -17,18 +17,13 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; -import org.apache.seatunnel.engine.common.config.SeaTunnelConfig; - import com.hazelcast.instance.impl.Node; -import java.io.IOException; - -public final class ExportsInstanceFactory { +public final class ExportsInstanceInitializer { - private ExportsInstanceFactory() {} + private ExportsInstanceInitializer() {} - public static ExportsInstance newExportsInstance(Node node, SeaTunnelConfig seaTunnelConfig) - throws IOException { - return new ExportsInstance(node, seaTunnelConfig); + public static void init(Node node) { + new ExportsInstance(node); } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java new file mode 100644 index 00000000000..b004e8557fd --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.engine.server.telemetry.metrics.exports; + +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; + +import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.internal.jmx.InstanceMBean; +import io.prometheus.client.GaugeMetricFamily; + +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +public class ClusterMetricExports extends AbstractCollector { + + public ClusterMetricExports(final ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + + // cluster_info + clusterInfo(mfs); + // cluster_time + clusterTime(mfs); + // instance count + nodeCount(mfs); + + return mfs; + } + + private void clusterTime(final List mfs) { + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "cluster_time", + "Cluster start time", + clusterLabelNames("hazelcastVersion")); + List labelValues = labelValues(getClusterService().getClusterVersion().toString()); + + metricFamily.addMetric(labelValues, getClusterService().getClusterTime()); + mfs.add(metricFamily); + } + + private void clusterInfo(final List mfs) { + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "cluster_info", + "Cluster info", + clusterLabelNames("hazelcastVersion", "master")); + List labelValues = null; + try { + labelValues = + labelValues( + getClusterService().getClusterVersion().toString(), masterAddress()); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + + metricFamily.addMetric(labelValues, 1); + mfs.add(metricFamily); + } + + private void nodeCount(final List mfs) { + Collection memberImpls = getClusterService().getMemberImpls(); + + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "node_count", "Cluster node total count ", clusterLabelNames()); + List labelValues = labelValues(); + + metricFamily.addMetric(labelValues, memberImpls == null ? 0 : memberImpls.size()); + mfs.add(metricFamily); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index 471a32c5aab..9f24f1ee399 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -25,7 +25,6 @@ import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; -import java.util.Collections; import java.util.List; public class JobMetricExports extends AbstractCollector { @@ -41,37 +40,29 @@ public List collect() { if (isMaster()) { CoordinatorService coordinatorService = getCoordinatorService(); JobCounter jobCountMetrics = coordinatorService.getJobCountMetrics(); + GaugeMetricFamily metricFamily = new GaugeMetricFamily( "job_count", - "The job count of seatunnel cluster ", - Collections.singletonList("type")); + "All job counts of seatunnel cluster ", + clusterLabelNames("type")); + metricFamily.addMetric(labelValues("canceled"), jobCountMetrics.getCanceledJobCount()); metricFamily.addMetric( - Collections.singletonList("canceled"), jobCountMetrics.getCanceledJobCount()); - metricFamily.addMetric( - Collections.singletonList("cancelling"), - jobCountMetrics.getCancellingJobCount()); - metricFamily.addMetric( - Collections.singletonList("created"), jobCountMetrics.getCreatedJobCount()); - metricFamily.addMetric( - Collections.singletonList("failed"), jobCountMetrics.getFailedJobCount()); - metricFamily.addMetric( - Collections.singletonList("failing"), jobCountMetrics.getFailingJobCount()); - metricFamily.addMetric( - Collections.singletonList("finished"), jobCountMetrics.getFinishedJobCount()); - metricFamily.addMetric( - Collections.singletonList("reconciling"), - jobCountMetrics.getReconcilingJobCount()); + labelValues("cancelling"), jobCountMetrics.getCancellingJobCount()); + metricFamily.addMetric(labelValues("created"), jobCountMetrics.getCreatedJobCount()); + metricFamily.addMetric(labelValues("failed"), jobCountMetrics.getFailedJobCount()); + metricFamily.addMetric(labelValues("failing"), jobCountMetrics.getFailingJobCount()); + metricFamily.addMetric(labelValues("finished"), jobCountMetrics.getFinishedJobCount()); metricFamily.addMetric( - Collections.singletonList("restarting"), - jobCountMetrics.getRestartingJobCount()); + labelValues("reconciling"), jobCountMetrics.getReconcilingJobCount()); metricFamily.addMetric( - Collections.singletonList("running"), jobCountMetrics.getRunningJobCount()); + labelValues("restarting"), jobCountMetrics.getRestartingJobCount()); + metricFamily.addMetric(labelValues("running"), jobCountMetrics.getRunningJobCount()); metricFamily.addMetric( - Collections.singletonList("scheduled"), jobCountMetrics.getScheduledJobCount()); + labelValues("scheduled"), jobCountMetrics.getScheduledJobCount()); metricFamily.addMetric( - Collections.singletonList("suspended"), jobCountMetrics.getSuspendedJobCount()); + labelValues("suspended"), jobCountMetrics.getSuspendedJobCount()); mfs.add(metricFamily); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index c38c34be42d..88b3bb0e0bd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -26,7 +26,6 @@ import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; -import java.util.Collections; import java.util.List; public class JobThreadPoolStatusExports extends AbstractCollector { @@ -43,11 +42,11 @@ public List collect() { List mfs = new ArrayList(); CoordinatorService coordinatorService = getCoordinatorService(); - String address = getNode().getNodeEngine().getLocalMember().getAddress().toString(); - List labelValues = Collections.singletonList(address); + List labelValues = labelValues(localAddress()); ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); - List labelNames = Collections.singletonList("address"); + List labelNames = clusterLabelNames(ADDRESS); + GaugeMetricFamily activeCount = new GaugeMetricFamily( "job_thread_pool_activeCount", @@ -67,7 +66,7 @@ public List collect() { GaugeMetricFamily corePoolSize = new GaugeMetricFamily( "job_thread_pool_corePoolSize", - String.format(HELP, "activeCount"), + String.format(HELP, "corePoolSize"), labelNames); corePoolSize.addMetric(labelValues, threadPoolStatusMetrics.getCorePoolSize()); mfs.add(corePoolSize); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java new file mode 100644 index 00000000000..25a5bfc42dd --- /dev/null +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.seatunnel.engine.server.telemetry.metrics.exports; + +import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; +import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; + +import com.hazelcast.internal.jmx.InstanceMBean; +import com.hazelcast.internal.jmx.PartitionServiceMBean; +import io.prometheus.client.GaugeMetricFamily; + +import java.util.ArrayList; +import java.util.List; + +public class NodeMetricExports extends AbstractCollector { + + public NodeMetricExports(final ExportsInstance exportsInstance) { + super(exportsInstance); + } + + @Override + public List collect() { + List mfs = new ArrayList(); + // instance state + nodeState(mfs); + + InstanceMBean instanceMBean = getManagementService().getInstanceMBean(); + if (instanceMBean == null) { + return mfs; + } + + // node hazelcast executor + String address = localAddress(); + List labelNames = clusterLabelNames(ADDRESS, "type"); + GaugeMetricFamily isShutdownMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_isShutdown", + "The hazelcast executor isShutdown of seatunnel cluster node", + labelNames); + GaugeMetricFamily isTerminatedMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_isTerminated", + "The hazelcast executor isTerminated of seatunnel cluster node", + labelNames); + + GaugeMetricFamily maxPoolSizeMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_maxPoolSize", + "The hazelcast executor maxPoolSize of seatunnel cluster node", + labelNames); + + GaugeMetricFamily poolSizeMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_poolSize", + "The hazelcast executor poolSize of seatunnel cluster node", + labelNames); + + GaugeMetricFamily queueRemainingCapacityMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_queueRemainingCapacity", + "The hazelcast executor queueRemainingCapacity of seatunnel cluster ", + labelNames); + + GaugeMetricFamily queueSizeMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_queueSize", + "The hazelcast executor queueSize of seatunnel cluster node", + labelNames); + + GaugeMetricFamily executedCountMetricFamily = + new GaugeMetricFamily( + "hazelcast_executor_executedCount", + "The hazelcast executor executedCount of seatunnel cluster node", + labelNames); + + List asyncValues = labelValues(address, "async"); + List clientBlockingValues = labelValues(address, "clientBlocking"); + List clientExecutorValues = labelValues(address, "client"); + List clientQueryValues = labelValues(address, "clientQuery"); + List ioValues = labelValues(address, "io"); + List offloadableValues = labelValues(address, "offloadable"); + List scheduledValues = labelValues(address, "scheduled"); + List systemValues = labelValues(address, "system"); + + // Executor executedCount + longMetric( + executedCountMetricFamily, + instanceMBean.getAsyncExecutorMBean().getExecutedCount(), + asyncValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getClientExecutorMBean().getExecutedCount(), + clientExecutorValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().getExecutedCount(), + clientBlockingValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getClientQueryExecutorMBean().getExecutedCount(), + clientQueryValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getIoExecutorMBean().getExecutedCount(), + ioValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getOffloadableExecutorMBean().getExecutedCount(), + offloadableValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getScheduledExecutorMBean().getExecutedCount(), + scheduledValues); + longMetric( + executedCountMetricFamily, + instanceMBean.getSystemExecutorMBean().getExecutedCount(), + systemValues); + mfs.add(executedCountMetricFamily); + + // Executor isShutdown + intMetric( + isShutdownMetricFamily, + instanceMBean.getAsyncExecutorMBean().isShutdown() ? 1 : 0, + asyncValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getClientExecutorMBean().isShutdown() ? 1 : 0, + clientExecutorValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().isShutdown() ? 1 : 0, + clientBlockingValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getClientQueryExecutorMBean().isShutdown() ? 1 : 0, + clientQueryValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getIoExecutorMBean().isShutdown() ? 1 : 0, + ioValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getOffloadableExecutorMBean().isShutdown() ? 1 : 0, + offloadableValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getScheduledExecutorMBean().isShutdown() ? 1 : 0, + scheduledValues); + intMetric( + isShutdownMetricFamily, + instanceMBean.getSystemExecutorMBean().isShutdown() ? 1 : 0, + systemValues); + mfs.add(isShutdownMetricFamily); + + // Executor isTerminated + intMetric( + isTerminatedMetricFamily, + instanceMBean.getAsyncExecutorMBean().isTerminated() ? 1 : 0, + asyncValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getClientExecutorMBean().isTerminated() ? 1 : 0, + clientExecutorValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().isTerminated() ? 1 : 0, + clientBlockingValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getClientQueryExecutorMBean().isTerminated() ? 1 : 0, + clientQueryValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getIoExecutorMBean().isTerminated() ? 1 : 0, + ioValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getOffloadableExecutorMBean().isTerminated() ? 1 : 0, + offloadableValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getScheduledExecutorMBean().isTerminated() ? 1 : 0, + scheduledValues); + intMetric( + isTerminatedMetricFamily, + instanceMBean.getSystemExecutorMBean().isTerminated() ? 1 : 0, + systemValues); + mfs.add(isTerminatedMetricFamily); + + // Executor maxPoolSize + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getAsyncExecutorMBean().maxPoolSize(), + asyncValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getClientExecutorMBean().maxPoolSize(), + clientExecutorValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().maxPoolSize(), + clientBlockingValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getClientQueryExecutorMBean().maxPoolSize(), + clientQueryValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getIoExecutorMBean().maxPoolSize(), + ioValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getOffloadableExecutorMBean().maxPoolSize(), + offloadableValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getScheduledExecutorMBean().maxPoolSize(), + scheduledValues); + intMetric( + maxPoolSizeMetricFamily, + instanceMBean.getSystemExecutorMBean().maxPoolSize(), + systemValues); + mfs.add(maxPoolSizeMetricFamily); + + // Executor poolSize + intMetric( + poolSizeMetricFamily, + instanceMBean.getAsyncExecutorMBean().poolSize(), + asyncValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getClientExecutorMBean().poolSize(), + clientExecutorValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().poolSize(), + clientBlockingValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getClientQueryExecutorMBean().poolSize(), + clientQueryValues); + intMetric(poolSizeMetricFamily, instanceMBean.getIoExecutorMBean().poolSize(), ioValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getOffloadableExecutorMBean().poolSize(), + offloadableValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getScheduledExecutorMBean().poolSize(), + scheduledValues); + intMetric( + poolSizeMetricFamily, + instanceMBean.getSystemExecutorMBean().poolSize(), + systemValues); + mfs.add(poolSizeMetricFamily); + + // Executor queueRemainingCapacity + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getAsyncExecutorMBean().queueRemainingCapacity(), + asyncValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getClientExecutorMBean().queueRemainingCapacity(), + clientExecutorValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().queueRemainingCapacity(), + clientBlockingValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getClientQueryExecutorMBean().queueRemainingCapacity(), + clientQueryValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getIoExecutorMBean().queueRemainingCapacity(), + ioValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getOffloadableExecutorMBean().queueRemainingCapacity(), + offloadableValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getScheduledExecutorMBean().queueRemainingCapacity(), + scheduledValues); + intMetric( + queueRemainingCapacityMetricFamily, + instanceMBean.getSystemExecutorMBean().queueRemainingCapacity(), + systemValues); + mfs.add(queueRemainingCapacityMetricFamily); + + // Executor queueSize + intMetric( + queueSizeMetricFamily, + instanceMBean.getAsyncExecutorMBean().queueSize(), + asyncValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getClientExecutorMBean().queueSize(), + clientExecutorValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getClientBlockingExecutorMBean().queueSize(), + clientBlockingValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getClientQueryExecutorMBean().queueSize(), + clientQueryValues); + intMetric(queueSizeMetricFamily, instanceMBean.getIoExecutorMBean().queueSize(), ioValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getOffloadableExecutorMBean().queueSize(), + offloadableValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getScheduledExecutorMBean().queueSize(), + scheduledValues); + intMetric( + queueSizeMetricFamily, + instanceMBean.getSystemExecutorMBean().queueSize(), + systemValues); + mfs.add(queueSizeMetricFamily); + + // partition metric + partitionMetric(instanceMBean.getPartitionServiceMBean(), mfs, address); + + return mfs; + } + + private void partitionMetric( + PartitionServiceMBean partitionServiceMBean, + List mfs, + String address) { + List labelNames = clusterLabelNames(ADDRESS); + + GaugeMetricFamily partitionPartitionCount = + new GaugeMetricFamily( + "hazelcast_partition_partitionCount", + "The partitionCount of seatunnel cluster node", + labelNames); + intMetric( + partitionPartitionCount, + partitionServiceMBean.getPartitionCount(), + labelValues(address)); + mfs.add(partitionPartitionCount); + + GaugeMetricFamily partitionActivePartition = + new GaugeMetricFamily( + "hazelcast_partition_activePartition", + "The activePartition of seatunnel cluster node", + labelNames); + intMetric( + partitionActivePartition, + partitionServiceMBean.getActivePartitionCount(), + labelValues(address)); + mfs.add(partitionActivePartition); + + GaugeMetricFamily partitionIsClusterSafe = + new GaugeMetricFamily( + "hazelcast_partition_isClusterSafe", + "Whether is cluster safe of partition", + labelNames); + intMetric( + partitionIsClusterSafe, + partitionServiceMBean.isClusterSafe() ? 1 : 0, + labelValues(address)); + mfs.add(partitionIsClusterSafe); + + GaugeMetricFamily partitionIsLocalMemberSafe = + new GaugeMetricFamily( + "hazelcast_partition_isLocalMemberSafe", + "Whether is local member safe of partition", + labelNames); + intMetric( + partitionIsLocalMemberSafe, + partitionServiceMBean.isLocalMemberSafe() ? 1 : 0, + labelValues(address)); + mfs.add(partitionIsLocalMemberSafe); + } + + private void nodeState(List mfs) { + GaugeMetricFamily metricFamily = + new GaugeMetricFamily( + "node_state", + "Whether is up of seatunnel node ", + clusterLabelNames(ADDRESS)); + String address = localAddress(); + List labelValues = labelValues(address); + metricFamily.addMetric(labelValues, 1); + mfs.add(metricFamily); + } +} diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java index ba65ded1194..ba64588cedf 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceTest.java @@ -31,7 +31,6 @@ import com.hazelcast.instance.impl.HazelcastInstanceImpl; import com.hazelcast.internal.serialization.Data; -import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Collections; @@ -41,7 +40,7 @@ public class CoordinatorServiceTest { @Test - public void testMasterNodeActive() throws IOException { + public void testMasterNodeActive() { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName("CoordinatorServiceTest_testMasterNodeActive")); @@ -85,8 +84,7 @@ public void testMasterNodeActive() throws IOException { @SuppressWarnings("checkstyle:RegexpSingleline") @Test public void testClearCoordinatorService() - throws IOException, NoSuchMethodException, InvocationTargetException, - IllegalAccessException { + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { HazelcastInstanceImpl coordinatorServiceTest = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName( @@ -141,7 +139,7 @@ public void testClearCoordinatorService() @Test @Disabled("disabled because we can not know") - public void testJobRestoreWhenMasterNodeSwitch() throws InterruptedException, IOException { + public void testJobRestoreWhenMasterNodeSwitch() { HazelcastInstanceImpl instance1 = SeaTunnelServerStarter.createHazelcastInstance( TestUtils.getClusterName( From 3f41f1e76a77526032bfa55e3f0d4198c8ac1663 Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 19:18:37 +0800 Subject: [PATCH 22/30] [Server-core][telemetry] merge dev --- release-note.md | 2 +- .../server/telemetry/metrics/exports/ClusterMetricExports.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/release-note.md b/release-note.md index 0ec37a63032..77c1c34aa6c 100644 --- a/release-note.md +++ b/release-note.md @@ -175,7 +175,7 @@ - [Zeta] Reduce the frequency of fetching data from imap (#4851) - [Zeta] Add OSS support for Imap storage to cluster-mode type (#4683) - [Zeta] Improve local mode startup request ports (#4660) -- [Zeta] Support exposing jobCount metrics, jobThreadPool metrics and JVM metrics by prometheus exporter (#5070) +- [Zeta] Support exposing monitoring metrics by prometheus exporter protocol (#5070) ## Docs diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java index b004e8557fd..df5cea73fa3 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import com.hazelcast.cluster.impl.MemberImpl; -import com.hazelcast.internal.jmx.InstanceMBean; import io.prometheus.client.GaugeMetricFamily; import java.net.UnknownHostException; From 75b2dccd95a189f09df4210d37c230819b4e14de Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 19:23:26 +0800 Subject: [PATCH 23/30] [Server-core][telemetry] add grafana dashboard image --- docs/en/images/cluster_monitoring.png | Bin 0 -> 557327 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/en/images/cluster_monitoring.png diff --git a/docs/en/images/cluster_monitoring.png b/docs/en/images/cluster_monitoring.png new file mode 100644 index 0000000000000000000000000000000000000000..8f534510cca1d9aa93b990fc9a76c83d5082692c GIT binary patch literal 557327 zcmeFZcTiJp*Dov{dV=gj*z_icJ06R$>03>^AfM$aOH~p73G)D-+2-3X5I*3>>BOf zS$4QXDfrsy-W&oUQYSli3-#b8s`PnlD6Vsfh|<&Z_0@K^UkSKvQ-p|AW35LoV$B8P z=>iuP=%Y*g?9N*W3Sx$N1OaL&fys=!C`SO3Z87I}F8pFQKfkNej* z?*sOY`-$WKnIMM+`z-$1uwQ>`r}}15-R>#K@cJJO`w>ClG5=_4?mZ+3{F`!+BkO;@ zbdB0K#iHayEKuLBw_2Gvs+Y_hrg5+X~w}G!}B?aUuyS0zPY{*J!*d5 zKJ~jFa1N`feIqv_Xz=y+nDT#k*l%iAox(LTMb$Yq@9SBIHPG0NNQ*nDO1#=pm3mOw zymFcPFeyoR=k$sh;8AR-JnodqYInNzaCID@e#1%BPn{{9e`en=q%#qI-=4=t)DC2;qyPnLj2KA#DsE?c>;Gq%fg5QP%z7q8L|5MlhsO6w~x^TJJ zlrv@OpOjx#x7;W6aac&lQo2=znCj1}`xbKZ>|`s5az06H!M8c2T8sRDSh?~FT*IxG zSGM4drlb8EroNmrr7;ZUbVQ>5mn)u!$Xe40C1Kk!Y58w0*;0O`c|&(=={G1|NIEia zR!RbLs!BET-QDqgFhNpC=P4W+KjkORc4ts_C#MPXIzwmU<4{&}o?$KX;g$^xEH8|T zY&#n+glgU}rMo+*gc{}S&>gGt*Qd!n*q-Z7lS&|8Uk>ETs=T%o78 z4k^>D(?DFdebsMSA5R)64*h%G=;MfGHYzQ8zTyH1nv=8xJS5u`H~hZeXD2Xex6wH$ zg-W)PydUHZa!m<1>f51OdS(<{4CIs0X$LY66_ z{4UXk3ktKcL2J~+KHcE&vQ#!P_YtLxo!wIP^%#K zxyNJzO+Og@&WP5Z4eE^ND3VFwtB)*Gt&+jv;iRzA;tcNAOW9W;Sx$N9$19Zeiq=3D z{A829aH=TaXMf6D{|xKk2a^y@xt*ZN5;*O>isMYrd1L6Ti{aL^bIX^t#NLFp5Qeh& zP!^1*mWK!K+1o^$YZ^1JeE8KSeKsruwB54XldL;tn;Ck^v|TjZ-#nF6?XoR!;Mr9OC;{$5 zO-GZlW}dURZUD!ktyj(0A7|Wh(tsSPILrG*tn=lM`5yJkU7oXBbYJLb#63u|oye(i z+Z?Y59~{Y5>g`Vz{+XvIW}cL!3uf`Zc8&HiNRX2}{w()$abvPLOKwfH+z^Z!{h|DOD%gd}oo#W!|5Cp+yk3K%`_>Eh((jv; zMnTT%MMx6upKJF^$grOC7XfZtqdJQ{3M&xx^ir;g7o*=5NF*?cuB&w`6iMifrI{Ef z9}330a#p>kq<<)F3v0>Z@>R&Zd(d+>S0xa zRd0>0g>f+#x*uKaU1lZH@)`7bzyZ7#-BCp+*uKKWHY-J@+_l$}rI7*)$g0u1oo~b4 z_zMjicMJd4f|{M)suOM(!2~q#(S=X#jtKSo*skVTHkOJlMg~<;XxtHJxiiPQ95na3 zNOSP*^ye#Fw;0*Jg6=TGKPGg?%4{@gtNUqy&ib3GgNW?!n$^Oiq4JYxScPXkjb4(D zp1#83*QTvWyFP9$s2aLd#d)&FL@pZFMQ2zUJ_tlLYPVocVNbRu zls6$OOLk>7Eoy1c-txEXo^MkR@WAtwsq{UOMl$iu%u6h z)>&rXu07+`)s4rc+Vzylbpw7SKT#NMhsM+S9n;=C9)X~+06k-1#h&PNo#a&RcF!Px zxzE~tgdZ+*fLjtU`^>e}d`62>Y{9K|KG6pz*q!mH5e3cU8ge~|Wd&|4=T)7UoN3NO zS006GQ!f#w9p=Ke@LglW0nq?60ipyHV&vEyG5Q*}skDnS7ORRKDd>LP`xVOM5vlLI zmeg5g9m?!G99KD^3-5hS!u*yi`mIu+e!c6HJe!D)kaolqD~+KCCo|v6N#IW5Y_Ik? zS^%LefA34*Xx^$DBU{JpcDbm@IAJIxWykt|FVnW1;#|VV!y<h=bbhTa6;0woB{_IxXf@&Q92O`N*s5<4-Sv!AY+M0yHmzma@7a5)x z^kIym2GCC&O3Z%6w@rM!fnpoe75*@!0jigaMWkL9gzZzLGqIu@=Zs?5*&R^4!8IL7 zg0#YrFu8pxZkbJ!?5xREo1zb1@WVxA__uF;PE8lNEBfAFJM)$ScP+vLWa6&ME7&tS z(lVD?gWgUM4Q2AFmPl!KDm;BbQT0w!>coitso7myL_-b#mZb8$&yIWjW{zuq&fDNQ z@9v;cEt_X-x9^vi&-pK`xE%3K%_{EgR)xUFWV%`W^VU`@6hQYrE`TVL#pbrL$3@)BTLYa=w>*@04d3#{ zlVqG_e~Rhaw;;6WTXC}(Z5#g~=H`6ogjbS{o}Z=97C|m(L@g{%i6Tiy*F<5**1S>- zA^rlaaG_3dHkAc6{ki8c?MFA`hhz20X#KFkIT#ZqrVr~&#s-YJ-aryJX;#0fa~MWNMgU{f{ zREoym1$a+u?*#}CW!|P4P<&$Wp`arJ7%K~A|NYuENa`WNRt%Dvy9nM1IMsJ5myw~O zh|V;cnLC|0m9>8je^)H)$k12k`ZLA1-dmN@M1xqMrnvg!j8@CVI+-^T~Dx}rLi zxyAJ2d?)4!X6s9muF=3YX7?z@DlV7~tScFPnZO^1Th)%~rA-I08@s<47HBM9Y<<2){FlEtiEl&D$+!;NV0wa2h z%95ZEz`lB$RLZ|RG2aA+y$#DSZ_LW!F+ZA_RHD@YH_DXIE<6Qk+s?9 zI6gNv}JP(LwUibZC z;d+6Qg1uE@c8u1To+vezGMx0>YHvvXczZb5sf$Kzx{D@SmDfFjZ0|z^AM2D6sK~wu z_H7W=D7NIbJ6@R3rmy!=dBStvd(7VU=U)C73qIsSzuk(^>;BArwvL3gvN{iBu3)&q zKmFd38EIe$CT|;U>dtoh(n6-&8n6URA#NJ04l4yC-e>-1l5!K$1-Yqjy0xj3sUaIyP{}3^a;1$ZVwdybvEti|zv_DVYQ{%} zmIwV|lkhJLDOR)?`BqkGbVrqUK7Z>n+tx@fD9a?00FE1!ok2~lJ z+AwBN$glDyYK?b>bb#__f-m$(9y&=_)AWF}dNjuYl%3j@%GloP& zv>icAo~Gs!YRRLyfOR#viSG!<+9*#5ePAnW8x#vrnn z?U~)=J3YZ&b9g-qpZg1nju1&@s}_wTShl40aa=cx8e^F)rbUv7c;)mgNajAUYxNOt zSA_E0oA(Wo5Q_n@iSzer9SDylVk1CrU@h6yZs4S!KKKH);25yhsVn|Uuw`C>YrlWW zRu*H@y|OXpS;1yj46V|Tx1Gx?)bL2v^kArrnJimb`9$^m1LgGIoq9E{E9x2I&zXK; zFmv8$z5+X4+2L9Y^V7D`8u5^b%+M;WY;Ze}<*t$3VCIW=$m@F^rZ zSj%I89AhQ%YU!gmCb7KuqeJf8t+!9)YK>5`8FqEkZQe&4*C|QNgkV%-f+fcJdY#h7N z>~0?k%KycRXyxw*JJd!Oq9s++qSlRU())WrvhXN+;a4W!Yd7>kCp81ln+9ih;8etf z6c~`imC?j<2d$}pqiW60P$+}M6b$iq}#NHyG;(W@y zazIq7?>7U0Gu^QwD_;iNk<4}%JIbkFblE;<<%_SC|U#X{q)xY|(b zzFOwXu`b5I%<(ucNOV_eJ+`58_NU$K;djsII*BrgfwbTi`av~uM!~JEE5^3vDcYNm zZ2v^~UentJbp7{5ra~u`4<{Y1Zq^%dp)8Qonu&wmr&J=Q`ME2&cY20a>iVux?f|s~ zoLsi_=sa!_fLU)WTOAIHo=o*kusY+|NZVv?MA@CRqAv^&SI1&@fF5Wa8(V=Hc@XW; z`9|+&U(yL}jfQL`6r&=&{fgJ)iD;K&F#EO847r!FJbT?78PxM0c3A&Um?1NMq0`}Lm<^V z12&D`>8x0U_5+;tqemJ(zT~q6d*Or2T+6bgp5?{$sD>4zIGf;@xsz zUJc3qS4P^IS+?Dq=AlS%|19J8D!HZmfTAiEncJfrM0XroFp{p}AxJxS+FcFNpu2;b zlupzi=-*SmUq0TlY=DVC3RP$*%s9{TS-eoM)A-3p_u2~C@*y4fgHJt!R>|L{_nWzB+^~ zc{nOz%_wi%kJBdFzMVCW0%iXiG64aIKalCSwTXrYPTQ0b5CynR9oEmmNIN=H3e})b zYwYH%?=eH;elNPkKn)C5+$P5vUww75;Iz?)vJ>!mJDk~C)I!?7G?`3M4 zagRDK^LC)!lGMasRXo}&4)H*KcKn_$8H*mLPfvLm9FRmSXyN98p|0G4N;$mY+Vxw% z6MQ?GByYlcf939m*x;7ypj6FAck!}0WrKI+$wJN)X9^S(g;yz6zLkj3nBew!gx=y` zvY%w;EP&8a_LU>FpVtaWwpbW*Eyl~P+hF;nkgV(zBuC4zpxE zW$|w^U26CYlP%mX@s#{^FH480PdNBp$p^(6dGv^JRF|n{y;1JBCf)8=j|;6QszGXo zIh2>~BNzNxRZaZrlU5J9P1IhdiIPh`b(do**slD^?36&pS~bU=_4>K^Zu!ctz-k2l zwL4GwDtkU0rVp|yrhM1z$h)?SG0yy`#N_>1)~d4Z*%-%1(5-hp`L$zfeTaD%7{E<` zqG0=uPhl~e6X~)eW=$`V$$gS*zH?d~T;9BR4qF``ix%FnmOz3~^)`9g{K5eofC^^m zhb=89hmcKR3GZi+=0G-Nz?((*B>@H2YOlzo$F*0V>F(_ZA7dUpct+LtZi z0%ccxD1*yDQ&!`r63X)hnaZsK!&$XSL&y3l#+3Y0alnZvy?w3nP)jfVM zrfx6WjZO7Ez1OjsNd~0gcCGsNX$*V!yIAIh!vmf%4za(QOA~)Bm`+Wh&5Kiq*qRsg zv;JLmAO(1<2?u-D&eGw6f6zOM4Rz`~GFL;MlyxecC5w~DEej$rJ19`7j_&2^DIWx=*4wL&L=SI`L=YOU_%v)Wq z?!cXYO*MYdB&GQ;trzq&!X1Jsdg6&oY%N;1PQN@DrRnT=JYK)Fl&@!neI~ zuW!nI|40UOk!Yb%=vvIkWJ@pJV+zEtJ=!X?Vb<&3tQ7LkQ1-!PiuH;bB7dk#pusnM zPN1zi?xIH%Qzr3=+%<2fx?gYuM==vJ#H0f+HFk=_kd=(NCUH{FD=4{MqHr${=HCL`G9KPHJ?nTe{Q91Fu@IoMU_KzPN&vx5@gs}IU8a1QP7T#!mhvLTp#@8 zzQ`dRNNxwa{-Y#W(EZ^YYvu2TQw-L5wI{o;a(S=y63XT4v~<07t~^;*eOzZo`tp42 z^IC)rhZ*fIzXCMbAHU-SY`A4Nyn-mPE2(K|kvmceTzO`ig0K+D9(-YYiBoi&{P=w? zmLheQv~Imln|g}wPt5+P>*U5BGk!IJGQsX{sdD`uO9>zR2l#&o7s3Oij-F0al5_=i zSC`M^(qOZ?Tv;;T=~+{kkWf&4Ip)>Nt`ULE^Q4JR+tiyEWDY%Y<@!qb+v3i`nD1{x zWAV&U4=hwF=V#|2t_{{usvku{asQ)YSv^XqoirSW1eFs2nu$>#fnmLJMkLtOkF$2p zBQq+jC5*ta5|FsZWZS#}Dw{0OTdnbU-Wm2Q~M$gT13ASJ5KFR2X_+HPS+Up$Z< ztA(@!f-;St_HA{&SS-D2KT#QeI*TZi6*hNjFP?G1yes*Bp4cEODtL{0BiuBUR6jhF z^v=1_lSg{#TYeOYF00ti>$ZkX_L2bITF7dkzS1(F`W-X*ambR~^XuKkRGN@(W&RMcq6k4=)e;_j~Y3F_> zV#XlFp>_I$YL6_Ixkp=5s>KqB@D65}^xnvRtJRWnB_k`P;wya|9i^P!yw_2ycNf^m zrLbv>!LhntLh&H0pRacQslJB?0oxjcTDZxA0^NtQ$a?k~92%t}j~;vv`{CHjLDJid zMsSEB61~Cemd03*Uti&?PR2IP=^99Z+GencPYK~ts9{13&Q<6^fMBLlqtz4ez6`=A z>j43&rS3F-BN0dOs<%hEQvATio^XfcqTp-yVer!2J;`SDyj0O9z0W!KWU1!@sOXyo zwHV$SJ3n`Pd#XC;2Hk^)0C(?T9{$!hA(~Qp4IW#ZO3WhTEiZTLyDtb3;o2!8oTntM z)FBzFw<)38^{MzysYLx0lO9TVX6cc;%@4?PVqp#Xzs(L`fBSyNDfIfR_K1DRGlw|5 zN~Hus!}C!=Ch4?>fQ}Rd@Dg7yJhRx;yV2Gj9%N>Mptlj3c@+r++XW3dm7s3(O-p4D zn!sJr_$swlO)l<59q0GV9yow!MTM(pIMiWc)`UAWiI|XP3uI2r5THk=N7$bM>`Hih ztk3Z#ce+DUu=K?$yFOOwQnYW~*W(S>@CA|-R_u*!l?@XH&~K(_E(RvWEqe@LO-G zyZv4(4rxY9wuQ?cygVIX&u*z*z+?|z2Rup#G$fgPHM9vUu9@_KUJm^_paqZ3L$Q^g zuA9EN4^QFS+R4X}RpLWJQwy0}E+lz_0<28OU`xyS@ERHuAd*8HQDf(+|IITy??M0X zCa*IceOqPgtSGzE=6(max)jIi5o`T>B#cf!BorZPS@97}neJk0@wE$n zU`qxDNQN8K$ZILdWz)Rd09)3ecEmwvY6c%Lw9IkTpHwM)Vm&jq89~65+nd=uUTd-KNOmaz$v`xG1WTOwZsxW+= z(t>h$uY9KahdUHgWx}c~43t?$dTD+*Bi1yr^;Ip&6Ku4w~hk{4AqD%63D>oLnu5e zAv_Pn9PIbZe~~5KUub3s-%wx z%wFf(Xl1!A2636iok}^X?8$N$+Uzt)QEV78yHtto%}y{(;UH;O5qhW4-Y4L&MDdW!UB2a~e`71HZ#w5=rQc za%R?w{p3f3(r98=hWF^HAK7C_bG%|@H4=u z+AI{n<9~6YRQkHoe+X2|1)mOBImkHD98hUIOk2{&ZJjrwS~z$p_x#j0*pCVf%i?GU z3{|Su2kf^Ni?24g?%_REifBvn3)f?qaA2#LI<#o|k1zUF7JirNOf{*lRn*x=cvqOL zbu_l~nxD~fQ#(ua^GE>u!jW~CzNt2^S&V6e+1sB1#X0USrhKVd`IQi@#bsVsXBIY!r?}6ykqIp9EvY=* z=8Rag7kCS?6@+}mUbPB7UK!Jr9p&&1?K@D=6xm>54Df1L)dU?Jy;WlQQ)5n`7tU39 zyb%{qDTkGJXhssZH_9(1{$jDF_QN5kgq#-H3Wj!`XJ|`jCe=jERF$~dUTspU7>+OA zy82$o6JiWatQunE3?dQWIvQG-xp*ji1GY5}0#-h=mq0eJ&%yw=f@qRWIkw2=?i_ru zt#%H8sx$k)e+oaK%4{dIS=3VH1E^knzP!;n7*BP=%_B}#@r%#v;w-ncV!pBL)YVKv zSm*@e)Fg9ZIn+-dM@S8qeeqsr=D6N1n+yRMUt@jI*@yvdN&-Ul+shKi1-}%fH*03ZQ=P@V&=^^kk>?0S)U!Gxi z1gyRX6F<1NyO67TEUMd$i8Vfb#%%U^<@`uMHc4BPiS#ZRdW#%=wxTh`!6ylOU{?%? z9&y2z3b_n)_qcQHGR>7ZT_)!GdszC0JWf72&2JWr zs2)CStkR}~{k@B7+r9T%>IQdQ%$5k7sI|GXJB7YNkM))ut?l`WMvHGRelJk3_CkOqh5{z%at zU)yHOlD3K;TqL8aI-C%5K9plSQ@s({dLe4<_Jjji#YjBEv3+S$cr<f!?(%v>YHDRCzg}7?h1&nLYp$DL4>^|M7b}CF@<@7Y8zk#>+I+|- z56m59x`R1n-HDEHt9PlbINx55po|}qz5F#FreDmMJZk)IK@;1oVH|qR)k6X2fy^t@ z@W0$0i-EbkF4*ZzxIrUXb;t#>+1?F-2{g`IlTpZA+-%437BNVA?B)&D5go44Wz(!_ zX#fSKsD-@_9Hw4;cik8q)n7=$1fQl#m9s|VQ>SfHQD*y{I)BfpiI4UXv=hPKu-ARk zp*8qknB{bZDSM5$Nvs;eOzBnmNyW%S%v9egXyxc=NNxZOuni*S9cM^(@EkG@ZOL8+kDs3*H^fgRZK z=!Wtso2c)|s-3n%GT_oMlEC6xejPfN7tlQl3xeJ#qH^a{1c_3f?Mx#Kj zAxc6AzN>QZbL)I|J-cPH03d)qdkt>)kT_jWKEPKp<(dRIBkl!Ql(?r)V&4smxf=@F z{Rdj<<8*nD%VS!hhK#^f>^Q}&;Sa7T)lJBXI}p!Vr)w7tzkh)N3@)w=HL6Y|AU@!be%wT-3W8A zX8#Z4|CAuV7D+#v;|-fF{s49B6rFzmguvu{W(iDxW~=p+Aw~1*5q02p#-6nQr;Fhq z6#hMfTW?C6gLhYH>h91N{HF(%Prd$IGvy%wULCj2XOO6$?9WBIBb4!NYNuaS{>Dsyu_%fqqD9O(8`>7tG3|dq9hVwr% zNdLE)c>Rs_&Z1TW`FFY*;(sLDhk-c+?k-<=XzKqV#Q*6hc_4wM`G2qfKPuw?mCygA zIe#7he<7bV>Vc?Kz-(;-fo$+v@dp0K+J4UMe;G3QPPRAt655>VX}nawqj`$Vn;esa z!(RORPKj6c(#fkWW_bUsgvUWw_K2+Hb8h~Z1#IJ``hoI{@}<%Fv&Mu>Lm=i(wfa@o zLRNi`Qe{C6T*91x0mXsdA@T{F>UrKr$wKbi+%JiT9@m})gS0|cD zhPe$=I9&f5+WC2}t@GX4#hXEKz{DtPex3z2Y{RACBdf34UzbSHySZrcelY&9$#IZJ z={j8bUt0Vok|3w9&8IPcFvu_MVnOU zZwZY5yBYeg{dL+uo`FwBQR+XK;{P5$VaGa^U|y;sqi_7f^Z);JgPq8>dxU69M9)-D z(rq(QXJZX)Brs*_s-$ZGn%kN$ffUycz8WRah4j~%T*sANVC0@;vto@){bT%3jWeY@ zQ;l_a|MHMeLvQg4ISF!~>Lsi%&3^%o6mn&1D*_Z1*{dwtFRm4>vH}(Z6pQk}=}x~3 zms%$BRnaaH@8W_$MJ1KVytTXGqs96WPcUkK^?&?excVmzLPyLk*fsl{R2;vGdZ3y< z38+E=uF)F2PkUyoJKomPj5AC%k~nYa099H=BrC$``c`Fr{I{p&eLKc=IlFHyz$t}mRYn63NgQI+C039hU)D%m2G z`3WJZ3^jZD&2^)6)caz_)0)3-QQmf;w7J@LO!s#rrsUNwL4*(qA{6jgRV9OGYn$0h&;{%(>s$CqN zYlroB(V1{WF^DA(Kn=K*iw1wcAft#qMM}LkY<9$-!hdX2i_DmB81qN)SJu6I_ip+k zl3sW<(0_s5bIx76#$BiO3Tk0z`$?y%sBjG$A%YgGCEH?SN4y>U=ad% zCH{uhxjazkiQi;{j-3SRtkV*+rW=o}g>&>CMGI1!ERV7t;rBWC}@?~j`2qUtfs?+`fOdbP(&|FQ>9(f5879*oS^ zbu;?xkDM|suIXig37|axEqY2BCg5=|0aWQ*?d{!bj=#Z87~#_?s>R5W`WCX8fV79@ zm;yuCq@jaNP-$7~L3kUhq{VMHdTB3xiQPquPdVYdOj7n@FGUyZ?Mehzz6{*^u}+cY zAnUxjwN8!StYMHE!s|Y?v6R1E~LPyTCWk(I+*w8( zAyv**>d-`FZJFemkz(D0TO`a*Uk3YlWvd_$wyjl04z->Y`yL_n zu;1RX&p6{)E6*!=29i-Oq-E91pMHA@<4ZgK>Wg|KbM= zqxWv7n}4`F5qwa~lP5npwj@B5KTB}Fd`me7f2mOYS9wrfw|&)##`P*X1&ds3S%I*hMWx(e zQ}?&YNvf0v8mH4BQ*RvdvNzLn!QPuI+bOwU%+y8#TNh-t-cMgSelN%h#s^K9Cp%7} zl=abL_T7}loYXF?b{q+45(7g`or33FFEg*&?0$&gW&Jk~Rnw;G{E424jFBHbrV9Uf z!9P3b2E|Y?oyj}*p+t)i=6I{279h(L~n!K3v8VuE~_Z^E!C&QP(wf0^p>`1~2UNbsmMfXc03P7ruU0B@1OJ76K_G`yN?{3k3ud| z`0X_d_in?tVh(h4^^LReLsXT?{D?0^pEt7n4N#Y@jW;N@0Oj+)b>*xXGhLQvTqf(e z;IeXmc{>%V`!YUxXENkPm(pIB+LAog^~L3L;fJPpa^1%nUUT%7J6Uk*zT^+Bx!YLg z>+EDR+S|<4c`epfWin`R?Yy5h?QUzV8SO9$OEVXNAQ^FtgtzvAF23e=f?6NmxBe^V zdUo^bd)7T%WG`kK>I54`)=s9|+a$pIjWy{`ir)Sbi=;eePcQ?a`p*BtkB= zIo7Rib)X2Ic}H zRdm8rv-C`zt9C1fG)9$}ihhv!-_$}wnNX;;JCQb2GA=d_%Y*A_!Y}G`9DAewh4t zURd5AD0?u@u7f(_?@ak?$(vKvt%ckjt+3#-vqYn0EPFYKdqL=!SomIpV9^rnw#guh zo$Qq5WAq3ucK@y)JdLz z9j3w66_;5X>o4b9R%+uPkCzX%@Q46Rf(blvBXoKXp?aAy9+nR(8xEZAJd)1e2`aes zM)kgAW}uY82F$tbi0C6$e?*BBm0r(w!Ow%Ut4iS zo6FrM_@t0NNif~6ocunUE9Fve`2=8Q$s$@VPeq2m`!A82p&mJ%u_~npIjsILwd_F| z?%*am4tLL-2?d3ro(mGBHF>92a9yjp?qg!@?kEN>NS5q7=$ywj`0p!6-#EbBWF5y_ z!pBSwF)6#Zx-)h2`_qcb?BpDr1!N9WuC3KkQG#tHhoeHj#D=b|Em3!Y@cX$J93+g@ zjR=U*hzqRrqsH6B1-sTvHDYf99Wy!N(ZRB$D(%t+!E}$7A9QLi!0{(rHreHp2NsR7 zGWs_ncFn@(2Wn09P>jw+$^_i|fy;dP7}B`l^%;RMcpO;B#254LcAK@ko8PMTYp2~9 zj4yok>UWoScAx7?*I|EYr;>bd6)|>2SS44lt;1Eo{D`Vb)xXq`8(0}BP#HuL;4$mY z(`fUe*mKVhD)|xxYWYfP{TJ-_?9NrVeSI#WV9(&Va4lr>r@OgTgSzN^(=4Bn4f3R< zUm3y(XeGbI?r0hgUUbIH58(`5Ji3*R=)YDBP-)7lblZqQr=mpwv;BVQ%b6=ft#^_K z|CH4%LybktDi%C2yJimv85*8VzWXD7o6-qq-2#ooemQ6n0xQ~ewjpw`LTfsn6;m7 z@kf&hxCZfVLB?n7gnhhXw(l3M)`ofg((H~v6}G&1*AHhYz!C$z-$CeS{iOO3^pm2K z&@$0Ks%zv-Gh<&>Vvidek{dw>`q{!p2_tPE-LL3VDS~@gt*NzZg@rPDlra%3b zN%30Y<6NGzq=a2#qKtlrqQ^W4%?AU$+B><4ga^}?K|3k`3bFLr0`xLr3`+w4^)7(j z)&*8kDi%ub4?3R>9a}YQO8a?#P*UbBX6DtAWcZEF=)2_mDGh`H@9gp(H}b>I*reD` zX!yPINln&1lSDk&5jx>BOhm&gzZ(2oJNkpKCmXrM>f_@i1u7mO2#>}_XY3mtS%-su zs$K8S?zq363ZYCO8)|&PaBa-h5VQx<35i-Vfka(@mhn!SS7Wn-KJmDEgyx`jKLNVJ2P~6&n?e!X=Vu4WA(Xyh zZKf^dD;|wKs%6q(fe71w`!7MFCMoJA97~J3fuQ&3OOTw}&gW|dkPNTn;_~x8$q3h< zi=i9G9%ILai`)Kdgnk!&mqC5H8LcMNw=Z{FT-s9vTRBDXBBugyhf>dn0EmWEw*`f@DBBI62WCsd^|U7RMuYo=@o`o4?l;qHs*8$|ZVEEQO)1z@#>rz@Kx# zDTUES&DLTa5`t^mp6BCbdas9mIM;ZG6kp0Sp|SpN=vJOTF!@02TTBewWf#~6(7#tl z6&*_zEu}-~Hj3nJ*jzrX#X}V=7yQY4OD1(|(-8+FPgGp*J;^_dDbZ>baUFLikh;tW zee$l_tLqLNh zKEV-PkI0El}3$747t>?!iBbj7)u}%m$dGM66^YMi%igdwwYQ}da;ks5;t##I-2Vr43nW%u?5=CVd~mLJeMT*O;au|S425&#{oS4I zQ_>xji=OjrBqYgnFqiP0IYOV1`$D*=4WEtDMcixvfxj^T?B;^{(Moqom9;+fg+QISgZJo)8>W9meQC8Ycw8x@}FC z04MEq&ucB!BV&9JRS&?}5{V)o{b)kt|Ha)O7c01vGV9j@>G< zcA+{|q=!MoCoe3LK~7gvY>#at+qELHZbQ_22PtNml%xm#-krt90y++K+Z=vtQ*jKw z1n!&AGr{UmzJl{XK^(ve>2Y_xg;Ll<^12Ah62pIn)vg-cvd6)j_^>o-#HPy4wwbMOZN{Q43|Ki^4)v$MCF7)zR#|K z^Aas*ZB4`RdG0SnD23%Q-Tp7(^iRgv5gqJfwR_87D{}rGV-G4vuggCWKW1?Q9N>3d2y|%f(q`GX=cNbM1&~44;r#D-WYm9j=098@Vml{;whrLQj63z?= zp~f43dn`4F@G6OD3X~`m4Llv8Pm#Mg3!2NlR?55_{jRnv29g$=XX%W*3dl1&C#~G| zjLd0VOXq!eYVe}084Ge&h|JqCGZ&u zNWXh-A?SEo9Fxuz`r))rf7U4a+i>Nv4Ynl_n5_x>cW2T!^LuTe(``F}3KGqOSROxZ zbC<5+j044LW{awEyFxJdG}|bseO7yr@e|rEGSi`ao>LTfEDJh24b5(u!r4s_ zGJPr8@P@m-Gx%~ark~011en#G?+3Sf<{UxreGXi2Mk=)RT6Na|s$dl2xc72@zM$!< z+M__?SKkyuHAY`7=miN}&A&QNIjc=#M$6`l`2-PA2wdd z_LE~!f+B*lX@;8a=m>wWc0|BJnM`mf}1 zx<)6SIGNbCC&t9KZQHhO+jcUsX2KoYPIkvda%#N)LOm8;JCafE^MCOp`logC zuVxU!zdWO4HJVBObA|uwZ`jw_|Eu58|9}4fZ$GI2^?ZUT5z8rT|LIvSl7YW4u_~!w zu&ZmcWntDsO|1=V0{SnC$O;A&WUmz9&r-|8`zYmBwu($-UU=-Xhq$3WS6h{^l74F9~G9uxCZ zDy)6(`D^>9?JTyPR%YqCT0}S4B-`WQYSS4g8{mmpRM4}PuU-9ynu0;+o@+kVd8?Y+ zB9Ny3XrApuull3AH4$eQ(BKiU2VC;p?{J?c#NzH7bSkh`t1hs(eBO1p*jLW&bqA&# zrMvW{mQk4nTsfuG1BSoDxKFmi7cZygHYCxv1~t|`13V*ji*UN(IKAtfL;@3 z=v~Ii>5bw+EU5Y|Q?(EeryH*-%$*}$6@N-5tu2-IP*~XQnZ?SylJuC>4>IgL(vRpT zw6&gpo@g#yBdNn^6X(m{QAelkPM(Vnm!?hGtr?{T*z9DsM(cjs+fJSm!M%6cI~py$ z`p0SuodGnA3eOFwHt8E#VS0-vA~~^BnS-AgH=+L5HNQ zLC3CX;up*IQJfJQ;s-_=3nd(>88VZOhsNDgm9p=^(AOq22dbRL2alg5E^_@ayZ~WV z5b#AkR{{{`!KC8`1x2 z!ZZ&*>fkT5N13Ap9Zj2w$IdJ?bl-nG8+o#G_|c+T!E7uFB21BnzUBsL6Gh56VI5yM zT|^)_sD(fK7LK=&(M;*w8R^t=Mr)elGl*~tD@5C%MD~z*9Kiet^MtvKBK%qre69V0 zp}y}p&8@FAban$|FB`n;5sIIBEYYn5cpW=VGn~NPI(fwVzf!qV8@M1?1*;!jI&aM8 zqLBXTI0n=?sb)oc=28MmT$ZtEzr~;W&mjo(r2y&9JVa_~vX9IU#$KtAxFd!cRbd&tGBFghJatV7D!aA->bM6cMti*F$i0d$+cpNJRR>k z0!PYK&f%f0{G-X{U%Pgl4zeBNUsK*At3?7Z`1PGG1tsennsOz~aG znL>>sW{lwM1J4;Y)<-L661ZS}rg*%C44S|Z@sPKZnIS0~6Td9DlI_*qNY3@Bp0C6l zn>jc+8B>64j0Z7(dxvryB?HWYKVk`m=@Ra#5fb3YpSn7Af7s zV&X9^YG}J-b{W?l`ngTY8K%9S2(> zYMVetpR9Ah9GB#UAtAYuAjP*yIX^| zrM$$eRob<@=){^hrNr!*BvUExKhSo5U@PbzgR{Q)AEj9%R8S90diFXTJWW7)yl!Vb z^wMM&*j0oeH*MIgXf0ho0V${GN@vFJbnT+k%`qoKm}5z4DC4O#8briCkc7hHi-I8H_>KwF`JL3Mku)8! ztAH=AjJgyignlrL{nDR{648%|@lY}PLtMGK2k|H;U$fw_#(S$ZHPhB^sfWB0HtW%* z8gblntE(YIh{J_t zG9z3uoRG#{X$xDfwufXy7_4LX?DZtM25X;CyGp7azMgGxit9+1=$5Dz^&bu-IZd(&j z-DoeY!%8_&3nk!HD&iQt3K`J=*4*Ma+0~g-n>R*21W2yIv&m%Q@7|%`qaBkJ#}fOQ!NyG1d^dZE-FqIS=V+Fk|^u7vT$^(`;g^$8w&5+~QRQs12E9UlEg64}RK&+>4{^z{Ay65@)IB z3TrU@xFYk`{^lam(o)*5=@s8(i=%s%F2aEE>p2TOsb8j{1U_7@G!k#mvme;M^|X>8 zJ-~D)!4CqB@WteWQ$%tp0!tOiZ4Gr6lCO1O!-%nmL*UX7F-4n;3TY%=Byifflrn5B z?b8L*^cdo7Lhk0au$7m}XuoFd*)CpIpRvoK0YTWgkz+|lVR2}-(zdZW%g84+i%)oT zNzcrGD#00$z*t515L&Z}^E2_N+@F)a<`^sLjXiP8HNRL~xpCaG{Or|@^c{rkw?5a7VRVVMlnoVmTja23}bvsG;)*=lZWh zLrXPAflUx!A3=615^i*ckTcH-#}3LeriB!igC-#p@^S=%2%)dlklKRLpiS6D3iA<5 zDb7SL_4Q%mp#KNKROw>yyiHRW|LuFyLfA@{2~0JfWv2#J_>F z34kE#RjQe`HwX#7|4o^0=ECE~YMom@R~<(CzW1UV^6ieoOf?kPnE(zU&;E#fkiZ zZ@Fdc&jVGIhqlG6lEZLuIguUmV+GsfyVfjy=5-`fkKGe{+sGM|hj%Ev&|S&jPAHEh z>z+MTg3Ep^s|0YQ3Lg?7!9Tw*!Vu^;j6o}HOtzsd1Ke>?m#`53tkTXU8|7;?u2j4F zxE$=2ew8q|So1WVW@Frjw@*7;igql|bWsIXGDc0&%dxR%6XjlH?*ht*B>m2uBpVQ|LVzF!cM1K$2Lv#MMKql#FB%Ebrqe;ohJdi zI}fwQG&>k07X1B@YDG|y!jZ7`^ zAS(l^KWE{Yquv*q4VnGg7y1KPtD_RBpPQ+&gzf($f;5!>bUb#IfBJI%U*V@Cq%AB1 z5j4EXSB|T1fTi1Z4H#BiU(yjFNqCbMa4UZ6q}0d$aO@oCsYNbU+vr=KQ6mTn>(I$eyZ7KPf}Tr!hk`r&y(S%h8DwHx>3@s68u6<-%ggkROh{@!`oWz}JaSm94+kT#MBHXA z&C74|+Sz9y!HF9BxizBFa-3v|8oc8v-cavcUg%Dwa0)T9;wqUjjL)Y|=(|&5sV~BX z6DUFTHRC(}4YI&h2TUE3>nu$r)x?t~ZDov3+J;#Z2K&dM)tEh8Gj6GXz2^wvtVj>i z5RjK!|B?$4;N_eFRw{9 zb$PeY*+irC(3TYgkgRps%VnBGRCRdTpomaV4oDm3a^VyIal?a3$Z>NgfKj8e_)KO| zZYkznZfTs5A+xsoV390sA4ZX_CObiSY_z)M4gj)%1@rXQ?ftFWj-At7PFsB#R5cIy zjrj&+A=W5(u8cQH1E-UW1pA!4H8lN&RNJAePE!%>JQ!gUQ9Z>z;yGBnX4;pg{JVPM z9M=6{xgdG9ti-4c?_;6Ba(ZNEwrg-!8M~G~dL)QM&V!9x^U_y1yV~QDtU1`O#cDG+y$VRi*b7kx0^24j1^VpuO`}Jqh;U zU={z+@gcYxpMR_!q7Kk(a&e~1pNnM$m4uwEOk%@HJ$VYCkIKZFHo=#2;3~^f(Gk$Vm|w=L7=Gch-UD>6B*ao z4#&-^S5dD;;O!mtyopr~bmB)nqh?HjX6D63Z>>*-Z?0m#vNf5_g8QqYaHJNc#x?GA2X+=C{z(zJc6?MQ##*K?!rZDgPnNFwh$ZZQ zs@x;CAljXnqZXmsPxaEj37R8+@XY?oy05^mQn4+$rLrn+h9kGewh9v_jnMtT#*B{z zT7+gH1?-mgic>iYSKqZ+d{x7E@C(?}v06cdTw>z3OJ+RQQIdu;gG#c}h&AoiWfZqg z`_ZsRi~YpH+iW9{x$30C#y2+T)+oZoB;!!K7WQ(Iw+X|Y=5LlNtF1NDGZ*+U~@ z91_Y?aoUCNu0#)-RD+Z)G$W%495C@H@IASx#ugT|wZ5tS?q^JY;(4YJ7Rz45JVkg; zB3UNGJjoB2yddXPFo%O8Pr~W+5)jH#0x_;h0nQn#m>kFW@FjV$)mOrZ-u;4Q!WJe> z(ON65Zl#d2&bHNny;V3{;hn;|L{T5sfFfH(@#aAL+_EB-nEi4 zkIhsIr6Wo8W$)B@(BqriOis1DDo4&l)Ki_5Cc8LT)zZLnLmVPMG>iM+UFJv8)t!B~Y*yq~o zJg`hJ6Q+s1Y@*ZETbXN7H|kyCFc&R>tOlM>sjY_f^53Q5Z@4->|}Zl~UhmX-HND(~N~B_OvBe$)7&^c$F) zT;6|ItGS3VuP8z}ME{eaatbvbjt`kdJ9yV=vuU)MjdwJHYw_Gd#XYyL4VqsO-;A%V zt9h|T_N*_&CSxtDvs@(ORK_^F~7fphGdRZ(uNsIU)tml!~Dt$ukIJbu^}tmkRO9R?r%zrTAVqQ z$`Q@7F|n*6eL1!!Y%1D|SM|IosU(+!Q&!sQq5OhWfSd3}uwR3Y9r9Zl>|Etgex~VT zwoyI`oP-ktRy|YJImE4zEkf0~Odn1n$U?Hjwzhx#+}_y_?v1FG%2@4Eo%@_=3FstV zgVBzge8*=g8?S+1F%BQH=#{Wisw>|IHs}=M{h;uql;B}dDJD>NOT2I8?ziY{)m?+% zusy4}eZ~yMLNYudsmhnv?Vv!w`47CYE*cr zv?6Byx&S+f*YDv$jRjlSf^IDvi+=hZ4sY7?!L6DH?~uxKw;j7f^$2h*10Xsd8Au z9O+te6=F>&S%LQ1x4#MEYihrF!9^G|rZFULhlq+7;95V1b1@uqXl1VjrxF{r8ZN|8 zaWkTUi85xnsJ%!`^y5Txq4`pcBq=O@8%i{T)%K|t5b-HCtEZ%G)1#{(asm{~4!5PT z%{&gEhTc)BRus7V4d{Oria5+(?|F-&4FP9pz*CQUGNHzE3a@Q(*8Nr7dy0#eq)HZ? z=idkPuga40(QhSkDWoN?70Iy~P>Ihu%*KTB*Z7G*o;&!7*r^S1jck0|gGGRnRofT!6!E z+5M;`|3O9-%PjrymsgDC5v+AgPyt0E z*({Wc(wcU%1)SyfmT}Li3xhJdZd4m*Qk^mTcx@UsL|hU+L?Sz(p8zr6Veq%Mmw5-P zWQiGdnk5?qJH^N>SdyU!HZ5Xv5dxdBze^M3CVk1~fV=3$0+}DM}Fk z(6ZqM!*9IgHIhH^R@tXVr8$wh$Crmc9@pTPY2{ixBC`mO2|LvH2ohFkdswj3zmrx` zZCs3QaF4i$A+6*&>PBqhZ#Nm8G>{9ZG47rD{ydAz6`DhcS++8TsCU8=Q_d+T(F_8T zQ;VjRQ;vOoSrz+pOBkV=ngFm<=6B|>PaOY)WH#VGz)g5uE`b9H$XUjk(5PZkzc@AL zoHu?O^CH+t!e1E!L;s%4O7mDp@HFz{MEZW{8BW~(E2Jy0Tji=ht_3oSlz4~E6N+Db)&-DSNHDPjIRKOT1l=EXiR?n-g7E5?iJuFavFcg zBAivLf3CM}t2&VEF@%$NFBd zcfBC`Eel83Yy-(U*3Y;G3LVM=l9GTq{Y|P19!%DOH%L+?gV>NfYRvhRP18ZTuTIW= zBZO@pG1V87r6nRl0Tq_av}z8<7L6;DVQWeSJ5QV2nm&_p?!|EJD=!S|IqC_`pqDxH z%(GC7MW8R=V>GL~b`$!rYg{FR+ehPbh*4DE(01b0YOs1MXPrD6nxEF&S3CWAy!xj1 znM;L$ib{aK!tEsqA=7*Obmf$*5atS1sad^>5NXmx>k8iPV|HG7e|7xQnh`E~#NFn# z2w1{60M2=ljz}&@t~0u}FZMGU$u~;|YKkjXz%KZ!%luIQ7Aq?V5uhL z7S%5mS{J-$K%(6xINCgCaVso7fXea#T*&qXU}E&^YKi9w+zu3q$TpO+?<=+3am0E2 zvN2U?o3=@6YlNXAVmy}lM?O&>vK9Y&%CIV;ug}Gxz`1x&5LIgi6 zXa#I^CSL*`w=d1Q%_fk&pVsqSD_qW8jlZR+lN_1GW`=r#i!g^)W|CR|hKEwRh7^Jn zyU`x{=$E;6J_LJEXM$u|`39~JOPGu;xd`9w@xmKlojvnXf31^;g1q<@XCRxYt&+AJ5znIq--N<5+nc;Mw zwpT-P9H-F9Ia=@~d`kh_ma~+E7r$3}xHIB9--PzzclPtA-Imx$#7k${7%;(NG&`_b~)YjAX_l)#>qiGq2)i`&`@D9tY(m^j*`c#Fesc#kW z*stfe3DNhtxwo0mFMwnA0O7ALy#a;#n-xtc9rL_Di91$)Uw<7x|3=iUC;GwB_sgzt z**7EQN!a9WQsNe2vLUD85#1NYF~gc#Axfhu%;H9KCH_X6*!fevdOitYszvsu9)EyF zmf$A*g}vss^@|}|Q#XIj{}1BZkl#VdnMRaJvKe7IsKhRete#NNnQ*N9yu55HaZ?=0 z?HSSdbk@SS@%lvx>(sJC&hyI&Nm@VDmO3B^)B zpCjM@m<$O%(UIzyA(v0)eWmHq8q0&V_rFs(Y*{ zmy~dOUp>IrVm8>f?f6>3=6mcIyGu~W65qc@P64qIsoX?1CIhNZEqgJ2KgsWlqI*l6pH3&B+wUjc~W=Tc^HLL)z=+_^wG6hFy3v zTrg3-2+hv>GcYfXe&Sk$HvVt%(+I&}Wx_$fFH?S<3*}=|0}FmSM}MI29_UA(mAqme zHtXTI6`B5|^>iDdUCCqti&0=#yyT}jVRSs+Tf0qIy7h+X@6f&YL8?f(X61KVnQqhb z#Kwh>gBkXy^5yR2E|fm^zI2KB1)gP3cD(i09i4(A6WojTgJ3RcIW=julN5nyLx<|6 z$Yi7kj52BPS)&XNgce*oCf4LIGKn9R)> z&-T{`vyze3MwohdZLZj)0m>AAhCX^I>@N>X9m)YQ9>r1fHXF}Xw=E9V%!pu2`xlsgG zIFG6;Esdd`^K;Y-!X&92-6^ZNrMk>CglIVN+fRk}<$ik22K{hF7rV%kk91cBPr?$z zb@2{cX|<}0e)JJm%X%AZSh?gC*cC;qPYX!HutfE=Csll`y5|P1pmKt{ahkzx>A71;ZLKE9~yKVTVXNx6t*QzD(Z%`2-{9 zn$+gYYzZL(9#xaromj9Q@*}%BCA1`f@w5uNMIw-uIgf=Q?T)dEJSP?ec&K4IsKzii)j$C6XFkF3n0!{udkN7CRu{?|4&1U;D| z!<*2i|s5P^lgkk4}gak2X-5D>DQY?KKZ!j{ML2?j0;9AR&++bW!*s%utUf3 zh4pb9f7;jJOFte3l?S|b&RO6;KNuD$7l0l;V@z`@c*(>(kC1;WjZvaSt1QDPd5X3T zDr^hvZn&6}@a-hJ!%q*ch-On_5LzixEpZ5};7#KLkMW3)0=8G}UZ(FRo)z&+BMhTg z_)^j3eYS1!wr!b@SoMdr#P29aO*X^B{#>^I3DkP$C6u$h&5X3@Q%c}1%4B?fNT0ym z$vTb&9f7*ebOpBa8vIzzQwBX8JA}HwL&jrLlh5#FBEPG@sIR5&UzhkQTx;~L*ZWs| zoM}Udrc+B==VILr2L_IJ%|7;#j0ebP@QEYsfaH|k?>evF z+boyS>W^*mjP9=;E6Gkho(}UjHJ2+uZbAafsz9 zLGW47NiLKI4@w)KzyEI^?*YwJqc-T4q9y?QdV%ru8&sp)`45AK%HZVV$=bI0#s`k_pu$pYy3zy(3c7G4U`P7d^H zUh69v5pqDQt0(Wvw@78Dy_P!G@Kw#wIMy&$!X4zrZdxjJ7>B7s2Uwo-5Re12HB? zcZ2O<(eoGK^?;1^3#S9XAj%WXe(8c1W>?gGqlfR zry*clF=bwt@!kfv!1RM=d)&T?qcfmz})oFFq++vjw**K+Tz zCpkTjkQ{>D$!=Kw)xxU(`hp3FRtdoUajT4<>q))`d(OANmd3=0BIwoJ;VXVCuR>yfeU!so+&J zriSEq8f-*&rbe?vvwS4cA0-4M~do{?v1_Z;rx%(W^_Z^8N|WIcEbHX>L6y z-nCEPWRe^Hi(`|1ADs5Q509B*B({X!$$)8%4=^IpGCzsCuV!&%Ch@W5&@(vwYMG(#$g6`e+iHhrg{Ho+=95g*+hBrLd z?y*Wq&rBSuw_&ZF{zc#{<%;4!9~K@Tz|! z$sqt4?@;&tn3y5~{?3x1OH-P9FYcyLDHve7IQ-tzxrDqox#8oh!25dtg?K(E;BbTH z*Gpl~E5F$-hrt!oOYhy3Z|B;1Pae}(TOsd+41}%z$#zd6tQV1F@}dIYh$qOzO1!B< zRBjHuR5vfOV(3ASV=vDk=*pu@4h1g)gdM9Vhg9&`{t5!IAwA%58*=;+-G_&5Wq3#F=CCfGT2d z?FUolgv~Vu2a^)@C7K!_ly|+pyYiDePBGBTNuM7CA?F&3{?kOei{m%Oi7S~qa{GwY zqfKoA^~I(#0?g^_#}P~FWgf)RQ7l(#Z?Q}*y~a%S^vNLpOTu#=9)A*HY!=xr)*;?t z0p!(0{sBL#-;=DXC)Hrohur|wSysDS3XXUw4UAm2Us80W!{Ks2YB%mS&Htod^VXZ2 z!h|I;F5I~E`^lEGe>Du9U#pvb2hKnA!N>Lxa9I!WhLONctFhiBY} z%wVfuF1ku6qH_1{+QCJ%1gAp)!wcoC!(IotKB-+<@Uyn~Z$;;L2ZZ8UOd;QT{ zP!ea<&TMiiTra-blZjYh>*b$c=$EVB%s4#rBy%*hT>$Xm>7`kpp<>xtfjeADgaB}Z zy>2=j`Nz4h$;?xNG~z$(CCtdw?ZewW|95%B;wVt(TioAK^$!qy3IV(7y(v6V7cawI zJdtCx6E$QF#uK5fu)E-=l}&tp-T?aR#OdIyb!?a3HNAW4$c^1F$YOqOsX36?SP^I& zL(gII@1D){yQ^DmR(ZF=;=rgRL%)QMu(m(4Rkl*f6KHR8;2PKHjFIQJ^&yOf*_u_K zm0tgezb7G8ud_BHM+>uvk>L0Jx=nB=^pHO-r8i-DE}^Yu;{$u?VNa)LZF|=?!pGAm z!=Y_r=Jr76Dw-)*HNm-4)nHCQa$4=dDJ7?7qfqffPMlW#%ES<#V})6=8&$2AE<*Bw z#+NR^k!0G^;cBw(Jl|ojL3i9A@mUuxnm>31t# zr{K%D!oh&7T1(&up7F`6cIk(TggbA5GO>Bf_4dPVx6r{Ho22 zACl^t04dUDb{8~(tSk|ie$N;I4xf4<3I5@#37hSbDn2pBe9MVJ+&&ZMU44^;g!CSdu6Qce<){pq+wsYO1nnf@j zB#@85rSpfJb?)RY9l88f772W(jD%q4J0bJa|-k3~zER-Z?UUDY{~s{XamE|spi zMcif%+L?e_s|SF*g5B`Bh1DXczA65LkuTPVsT|lPkmX_Gju2nN>#x6ojrJ52F^xLI zx`VEUOx^nEB3squ`hlup^k`x;Kx*ufp_5Z}Nl52G}i5+#J3M+#xwrzJ2#?|28<1Uqx) z3*3AL%FTIfJ8Hjsxgftw^>=KpDQ_br`ZFUT0bDxe$DLHh^1>a{@g{lt}#3|hu ze9h2Gxi{U|#-8uhZ5I%tFby-)Qk2141H$cF$?3{k@lzprwdO%f+`It%X%lj|M4ait z;C7bnn4|6^;+x@X_uuA24cN%xMh-}bm_zQv!3P`+et(Y#NrxO4IfW>r#Uw*-;0Y3s z+_hdGktZU~JtFwF+2j$EBemVX5z|y}Qj80aJ~zK4C|sDWfU-wE2{&DX$A=)3{F|}o z!j22PvuR#08tTacWm?(v9}dnF&;AGoN^SarH&cxV1ROx3+;tJ@g|1!f1V0x-X8>P9 z1w>o(7$!tfpBqmi)-RfL#(*h@n6vfq9>#`Y*z4o2X@8#H25!0?NIFqw6Q zbM)#o3OcTqXkfvN-nFsT3}wqyaN}NF(B@M%M)WbLqjMr&u4Z>=Rl@AD8sgI2iTR>` z)Ub1-F*8t)djKZSDOG=^6jg!F5+S2MHTO8YCg{sMgsRtojO0Q%?g#K;EWCrFdnhr8 z@5xD)%zpz_0}G$pr?!S&dT1ltPCN`8abaF2F6F|F_N@H;Ik-F`iluaOS+9lC`X4WV zLEUzHQN@RhelOciu(`0rqO8Nz=SD5(KUHY>nYfO* z7A8_T^l8 zuXet~%#^QBaa?;=tTcFHV|sgT#RBk2(G9duH;W_Ge5P$u5*8mz7muYy(g-8d<{_+5 zy%@VUtwdhHfFn@Y^$BucP~8pj&e|?;vy|hs=wiN~({+EYoH+}e?`hIR0^wGcDGsk! z3CkhL@}Xa@?J?Z)o$t@ z{@v4EIf1JdKFC%Q`$8piT5LwSreOp9whsxdi_U3l^O`4P{@H=@Xu z?O^$NcTENHd}nNjMQ-B$wZFTVtb_{5gj8X2z}#fBd2 zqlw63+~A*O0}KLqQIfQD)Pc`I6unw|+~xk`ZpD3z4zlMTSj1Rb_Izf4{eVMI8d2wl zUD<-;S^rsLgNkt`Ra<7z_{iStnq?%$1rb5!%Upi~8BZQFzCKU9T?(UEC>_EPd;$T?0Cj=KP^`;7}f*FnanCsRgg z$w<&Vfm?pBKwSsXM_c7eO})W5%1S!o{g89U-!X(znDbMFFK~Zd?2GFvg--*VK~uz5IhTh(IPA`YMtekCCZxP^7o{4qez!Y;;B zur2AQ)I)RFzwg)SwsL$v6_{POQ_=a{qA^(DWmzJ%Dzxg^mhDYp!|f}=v<-|fCQA*J9-sL-b|vl)-ixt+#@yO@^n zSG?xJF?q{Xv(=j{5#&F87e9^L{>(y_c#pKx9TxK4VI3G~+C;_?M$Nb58#j8@|I~Bp zHK}p84cbN&lmhYEpUavc4Vl^FJ}K1??DKk2eP~;9%@=t%t&W(a=o0w~@{iczl&kLV z3pm6=@9j2pNtT7B=mHInR=P&q5_|=Ytj@C=EU&p#3m(kkG?tO$;CI`+)>~2&j@E2` zQT^!JS7@1aaxiQ~l}DB4 z%qhNrlDdo>oCsfTd3wu%JYNlx0WdCJX80~u+dqr)0aTAMR?jz&+&!PY+Y=@ix+17b z87k#qg-dX(eD2G=?6J|){@=z6!Gq?B3EbmRduG(4SL(oI8 zrE3+wo4FsuQ1XPv%!juu)t2p9NkFmfS!m78N4O?HJ4xty;&>kxL#Glyg>h%~K7Hnn zn_R(BD-owV>RApj&TR}P$uJtz6};6B8fbQbYJFr+#FPG|5rxT+JzPBtz6!G#QV48{ zku_EWFjhoM=b7&pP6|MCXk=d7JyKcCQfVjU9YABdBWI9%3o5rQevusOjxG!Y_zf zx6=hc45O4-K=p58QnRA)6GKN1f;bLiUGUHbEE>h9UcNLusYkp3yAoaBg|%E60Q3Ha z_p#l3G&i6BD5tLnzPnUF>`$a>&z7~c;r5X)e(&FQhEdh&pz2KVQ>Px2;~5pgi1L4h zIf!&I4FV_Y{^0jbRe=}+V0NOVi7+|xvuv1u=vyv#49Al`j#Ir8GOte48+WR1doS5SN_T$H^9m<^Vh(Z-co^) z=CmI)Z6c*MxbvryMMTDvNQ$>wqB+;(JfQP$H3&EG6!#n|udQ}XX8}8o0gfI`vP*-8 z9hYWG<$~lwg2;K)>AORI`@P}Ln+3t3Y5YVw_I*_8>JiP-V*)w)+W|QKGKjh5`xN@4 znH8M|Y!&IDejCVx2TejVhwsbxkwf{X8b|irDu{b+mr1Xxk_NDB7h|42fFuDaI#^~_ zpn6X0W8oUUQXC$xDs*^$2|+Kpev-1d8cz)(o~$~T=nqv6&s&k0TecD11PNJhrr(1< z&d%@w(#KMNQ_1H>j(G6x=%fujI{qp_4wf-+f^8hnE$hyEMvmZKGf*9md0NTo+?>(< zRyBo#Dw}9j*QxV}s!@(h#iqksMs+S*CoLNuu(dUWthUS+5V!VBU)jSc zb%PxD7$u;+?gv^Qge4SlA5D6*p>-Q;26gV`SI9SJ{Irf7to-q zl+}9o!(S&Ax{IikNqe^nAtGsFRmSkXT!MfzlCg8wP;eVG+4F4{hc-c6c z{s|5zv)u67G3a!&HqK#9 z|1vle=G+6n`Df%i&G`?tcj_LA2At=}=X4yC5vU9_^(o~$jLzW6Z&n&Hrte(Fseq>d zo=XU~@58aGUy4*yt`GD=4!Upaix6LzA;vFHT;tt&4M64}5x!KrcX8mi=#}HW>%Me3 z&~v5{af~ypbo$$sKDTdwM+7*hmie~+ntr*Z^D(yAmi{GwE+@#kbhT)FE7mOW%R1AG zpRc_nO`$I)8B_jc+!^33V!yiOQ=q%Ro4+V1Pg7XM{#G4kYx{F@Hu+}Wtlz6TL)BQ5?c0k*N|GOw3I2>#0Q*un>g23av>!O@F()fI@U@^LLIg5 zBfW0<@OQfrvOu7X;v3QY28U?=Tf|9W5vi-g+O>;hyGx-4NJ-QjzJ(qjE=N|(tKoOk zy@S4b<|#Ky^pLRjThuFNo;6rx5XS zx}yUDJNLOP9znW_`UC}6;|2P8|H{c=O1U0&ueF673A5;^^ZY#D#0-Zamb{(k2yYj;+~1CSIrET7;tn;e|ag9<*iAzZJzNhWFdXPzRe z9F2;6M+5TpB6lTEN|HCM=U4XaPu-{t8U_RbUR(433$o1&{|mBpjsVSF{()=|(D!xD z7mJf2fwqFV8^|{2gBxU<=tfcMITuKXW*lwLIU;H6qr2i*oIxZH8={^lccCu$nFo&i z=Q?lFXg6lL$ zCZZ);qIV(^UDQ#Akx10&5=8Ia=yeQ2h)%SK9t6>QA1!(>F&KweaA-Rr%c z_5BCV4`-cy_THbijlR8ZG!r^`B{j56{oZPyxH9oRiqIO0~$Y$A8nY~)!ZWsOiR## zCG;hey6tt@pjyvG9XHYledKs&Ng}LIE<4T+7aP{IJT`1*KcZzB7YGWhs4btK?B01~ zPOHkLwQZvhAW<(M7p&wSFOgR*FFa9aLp#IBy^j3AXOk|830}epfDFIvd8p8{piGG0 zqU_c8gU`#HkV%9*TRqckLn6BR+HBF)c=d(*dHD&eSZa!3Z_#xUeB^8d4nOHojK_8} zeeR@rK$l!(vzXzucbCvc2Ng5!YBm?0cr$M;?rf(MRam^0X>jEV+SglHIi~U)N;uGWeDD)NqH0) zIil_R83*51KuC!xI0a&#-C&aiv{boTqxV6Rvn@C6)>{-gx6_V(Pfk%n^>Zw@5`PU*jNmAFCy-c62%bQR#E4t#K0+$*dSkmk>b)Lf{EWD0%&;g?UB9%g!|kk%Kv|7tZF zCclh+`+X2Sf=cxV0wOEb@L!Ck96^?*ZL#*?FwW5!&ER3i(y6fwfs|!Z)ON~0P=J9j0JS?1y{nNiifUJ*XOCY@4>r0bC^RHrnJ5dp(e^_Vn*!Qf-@g0*^+bgia>pJM)4>8{=s7Z#H**J`m(@XE<*s{;O76Tl z9N-RmIDAomw65ZKmKGhw41bAO$E>&rNp@FZenVC zBdiOL%+-fpmUCLSIdoN;^%IDqZk}+J!W~xU5&!K<8l)mbpB%@61sso;Sy@i9RD?1e zBYQ|q8N-BRJxTiQ{pu<_6162LM%s}J47z0>}17Z`zDfHw7)FN3dQ=N z;`zM9%LP1zE~+m&Q+-jrz^53sveVoggk}&94g`0Zt*@^76#hKRPyBhrFu?SueaKw!6#}&rci1Ma|pev_e&vhqD}MeN82sr|5+=#hUNV1wdY} zzgbc-g}IAqFbPkIRefkWubRht>;6dwIdSk zJT)Z3_0u-^lDXA=lFV3W53{efPZrC-33lypF-MNJ8KMqr_Lv16%RQ+6dR_dYby>%g zUhgE2X<;1*`w5)EIs4fX`-XUcRj#VqC4=B7BlZnL%s$UKp^k!4Ix(n5m3eAIV&O&k z1F$QlB-i+JrrwZ%2|2z&UvTRFz9;3w>O)M_r7qoL>p8(ke@hMv-@2Ua@(e=jDaqo$ za=uEkH9JhKBO}(b9inLvq=ChZ|GMLOsfM*{#LJvhlt&-D1=>)IysUcG&00D;dh8?B z)7(gtj*X+7svFN;b@b)O&-vBgf^-Yo`pU+P4ZltD1gF~8kW`e*VCe(32&ddfyF!a}WZ1 zB`#e_$n6&~NZmD%J$o}W*bg?lc<`n&BTef%`8whtAQV0(hgY)VxU88N$lVr1%0aLB zYx8p(>u5mCATksc`pB)~I=|J0%VO1e(smF@FYXk;4_>G9%5O!H`p-n;Qoix&E3Lk< z3h)kqn%|y8e?}jWy>IPRUEO?|8SGr6_Gr11Wa$I)_%84OJwjX!M^de)_k~hF)fdBo zj!YgdQdlM_*_uxrpos13Y*(W*NqZkCG*zP@h8RPadLO>LcH%qH2_SpkFBOk8hnfCn zV2{->{FECbfoTMPj2iN`2}XATREc3_q*Ki9W1qR!OCN}si(10MU%>s%Z2bdIY54UV z35rqz2VVRkbew z>PR@lqvp4q(_H0QVPecpj1E$sz|Y@GpSaC*N%O&D%=DT%^u!45=t$Cr*CrO*G{ z8d8|zJ30|35)+Ukij_FjOBaoaAoSOFU|y`P#z%vrMr@=WfsB46bRYJ>9233l}2IBxKY~ z4=J6i!}1S>G%Q79hVsL)*QqWB;E!b|N-Q()@;&w<~sZEv;adKjXs~FOW{Z~ zrz*lUR2IPX%=_3DP1arhf+|eAbV_kJ>8ZVHi(dR9- z<1g85#5sZQ^{*$}{w6YIGm(@%>`wbM_}F3m6qHEmD5%3y&VFXqonoR?SleBZ%(+MI zV-5fB$Z0EJVraG7vx=|H+b(glu2wYy1ro39>HZ{p0=JFzx({jCF!EQV=P}Sls7VIU zoc!JpJH9!pAc=JP9wxi20wg&=cDng7fT9szONg5d@7mlrbip5Ef?7t$5p(aT*QVR> zWv1qal-~C8iviET#Df$c!%RsMh)v#oz$hEo>5WsrT8npW|BaegjsZMXN}^o3Z|KUu zh3>lv;oGB9dGfFEzs01cKIev?#2$VAxIET?7G3J(1oenb=FS_2$L?~r>t2L0 zE#!z_ z-PFk&;9*QLOkLtE@wZeg#`Ic0=0C%#joy`Uanm zqU{Fdso2I>h@aNNe-tvm{g#V=ywq?|By~fTl}GXQUa2VBHh>S@&1Uu5W~r|HQbs>L zp&`$Doa!=94LV%bC|>#S<0ag|TDgr~SsqTGA_~Mi7{^A17sFOJi@_8%t5)yC`L4^j4)640x{Yp2FA*RmxwB zw|&r9ZzA=U!F8TrRy0k?zcF$o9N-(woWN{)+Wd?)>b~q zvhB$fFV!c@E?F?%YtoB(g7MI=qFQ4{{lqj*aecAv=Us9t%)Vr^N1a;U^tW>)i+PU& z7Edbq1%?u$|E8c{x(g4I?sW!b=NXR-BIk+*UqFNaE1Tz@u*b?;K846a^%Y$(RfGTe zqBnV#@sgkOQUP&`LM-(IcBg_|^H{o$j4?P`C4hOk94Mds;pk6H*g<#N1?B+v=gl(M zVeQWaBMg~V-USZBsd{Ichumw+n#Ljz+Z}Mr12E1qF^RWTV{hBrJD&4qKDKJ>u);o? z71<|b@~MoLPm1HJ+y6-V*y9N7H36)dO)dRXnMjV}Jm0Yb=@XwZ)IXCGRnK7>E_oDo zD6h{zp28Ra_NU2|G>$#N23o_{ zC2^Qa6?+L!mFf)TYbdwVunkMP5!{xB3G}_(u-w|;`>g&Qc9|{lB4!jNvG`+@aV_WZ zQIZ<^2Z^ulCTv5BIH8;lw(=Zfa7Jf;xPl+w7%OJF=_)n%x{oIUOgV;MTglJ!si`GY zF?#QbM9~&IsD27+XFK@ep$Nqk>!GuD_X{cGJq9-fx9JuAfavh~4U*~|xTb<8yzwxn zXQ1D9xrMPJNxX#R#>7XYS~2XAn9CkV8RIF*BRelWFb!3ImGgSytFaudz2fBWKT3w@xj* z3wk59SUe0Xklj+mUeYJxKX-V_v%Jn{TlJufqdkVMN6$$C4{c)aR2u>v77$OzZcgIM znOqODxrw!eZ#fRz^ui_=!oNO%e~#V1s$rX^Gm^ln+L4z&I}$4SR5w|o@8ENQd9XSG zGp91Xz$UHI$skS*iM@fI-GU<}jH*kRSH(x`Ug5vpLUGen-R$x?e!R^D1)s-AAw(gm zPFb-T?GGCM&nWjb2pVzmmUBV-pDrYLC}k2zF9c+1BrK{I=E&7T)Ms4s-iL+Y6Lc>3 z;Ks7IMz7Qhph{(=ZA^lnJKlI^-`Ln2?${N?m|6^0?LZqScplZs&h={kJ%`6O_Ko!P zROgI*sD75IjTNwIePN&G(#e)Iii214#F4AwX|96S_CfEJL0FhJL`>tiTKs2yf9t=H zFmBEP`rWAjI5krEvnoSX^xe3quM&EM4Xq4$ED45S*sy{903M|JhzET2vGOtew=Nu+r{D@fG#87i!{T743+TnP;GiY;QnE2te& z8KoRaIYSk%{vG_*lxM|>n`Dc+Hve#~2SB+*g`PYZ0C{j}r4XE7K_;nyTt+@@@+jot zTc++Oj;1|TEH-L|-PPv${J_+e=ajn~AAUImafI;1EEO|wa-}!bW$A7S}Q&kh9OS1XlomXR5e8c7w+_zxGpf#7b z92bt8u{3+X(0EP849qHb;=!@r(nXC_%Zah8v95N(W}QFw<2cvc``F4%$R2su+7+9&N4I*D zdBoWtYNNzAW8sG!h2xr1I`QnH95{-!N=fOv->xlanhm{Q40{;zQNYh}#_Hax7FUaL zsiqMRXgUI0Zv~=CSPYFO+WpFE41|zn5(rzIiaOMcSRyt6&|~^{-w^qcq4`27R|Osm zrJ6ZV3=+v?E1Ex1dR^x+pxg3NXrDGgsK7pTx~|0E2=(BW?BJe$Mr9x1cfcR0$)DA_ zBwfK^KMgWiD0ZZvLn=pHk=*sYiVb5PeN4mkzW>siYQ1ts(%evVIVb#M&^sWtzziniBK z?rX|z63@#hygb2&t@EpgIQN9t_tJUQ!gPc_E>}$w;DCOSd9!2PxcKlH)Q`wuzbuj8 zwfU%ScWI2|1D7ohBltm701*JT?n;-&-;Ey|cNzVW_Sfh-t@GP8kg;c;%=Ro*e@y3kM-hj$^x&At>jPG!Y{X1 zpq7iO0Xrvg_y^`ZurSo2({=25rnZIWVN~1?X_Yv1eCQR}A(a@pWSdq#PcYT8M^70q z1fDG02^N@PH=t(XXB|Ys@wZ2T4$s<96b=b%=N=h3A3Nq-qSGcf^|?H+F99sLVF@>q zZ1pD9X}1`U|>`B zntQH}LJnE!PlqYkxwrRBmsDzZ8f2WT)Kb!M?UiEoNocwY`a^dBExT+1wFS=u(x)8K z{A;@TruGT_ZB}LO=^&<8_Zb*E>=aUMBRiL8Jqi;Vb>igJ`o0Jfj)Z$S}sM|35}o0cYkY=>Ji2 zJ4%|cZO=o!oh);b(mu>SrJE(D8gP(lyVs}&uziKNXyK-jduOYQNn>GxaqG%gMY6(w zzq#SwxtO0;x3k%;@BaHZaA$dJy0p%V+@Fc$C}(n`-1A|68#BkN$!=!9LfP;f+zBcT zEf!9viRnaK$K+`CIYgzBlXk}_ec0Qz7ixf&>Laz??Ib)Ghm_xkqY$dRA7_*hRH{TRt}Xirv2DK6s0I0%cV2SJAGGV6GF7!o2>lp)3Uvnkv7=xf?)? zURU33Aj;39L1(9cyL%(uUir24-K8Wjh51WRfcHF#yW(7ZvcutStr?T);#1rcEn2EMbK>e_)u zK8mtXl~Wr5&a>dM5&{4bJv_M2#IX5XLw9s)aHm@tJc{S;!eCATFZB)SJ3S5%_EYJO z)dKeqwLdX)->{_3%7C~Zq$AVBy3`oS3wOfvuq^~mw<^ljJUJs$fW+w^bycivOn6(A z`R&%!uh1Hb2R9rb#xQbH4qF8;oy>kOX0?^=y%v=oA}?NJJ>(E>?0c%~ygw)5jb0g- zwV(?jaKP!=^+i4C9C2F#1))Kg2e)1IZI^e~%uH&tUt--YdW0YfSaGJr^E;Ch@F(P^ zk5Uuik#R#e>llOmf{ier%q(UEa^T+jSbi8-uDAw=!C9trB#(Ia!?@&z= zHJAoyju~TtmV52lm8O5}Jlk$}*6{i@1A}Ei6-_{$+wR%s#$79w*;b*rMdSp2#KKil z+g_IU6>-1@((h^uc_)vAT~LB3d+Y17Zt?F5dqXjrX)(!jO9*f3(b!WT@=uu7I2zK6 zoS25)xrBp$feX)1RHhba>Jbi|oSMqXE2>>Lf4#IUE2ts!h()wcMSa7w#FHC^McF#j zmpHPUurDY_h_3`gvlIY z`Khs@zF{ol|LiiWj>rEx{=;b1Gu>7J_9s|DILZVOMx5zup7_oX!r9+HHLh=;Rn}l_P zvuzggpJx)fnJ#(JF-qDNWSq;AvEtXxpVPYppF~Su=UxVeJ@^q-P?|KBZV@}V_yQ>M zd^8FrR)i((&O9p(8fSnC`S*0_AWPVE&ArYA%(1cTZ)OgRd(?j^iMDPS{1{Cc`Vy-? zE63;W4XfR7S)N52OFa57= zml|S;>5EItmw%)05&yB_Y7zIUZ2T_-0wX1|L15F9Eo^+h!FEzQEOg%kdaZtz6qzO1 zqR$-#{{SJbHa0OGmN&Tlyfpr95F~-Ms$!&R{4ddn8u6<9-Y}5Zr5yZdBmR1fFTL-4 zawO(RErSpEhxD>^ID6}EB^E-M>3G1v$XlcMZGPaMVkr0#|@S77+DKMx6l=+;Q-?}v@k-lg8y>dZ|R{r-cWCE z&nIEBW90V_rK=y74jmUy{5LXfcPv}bU(7TYv5 zIRZzEduAMTopp*cKYt3=XEu4^z3!NGQ{wJ?=?v>p^K^!6QFqL6dI0ex1G;p4s*XQ; zO{X5AUWfq_TuNV5>cMHPB_~?WGk50Y48yp?dJhX!+%=N`a>mnPs)!cOrptr*Oi1Y| zI>Tae>GnGp$=B1potl%KF0)=%8Gmza@T;6*QuvnRE!R~K2i!K)Z({-5ktj zex=YZ-Sj?8mVp)~6YmjAzhXW#z7h6wFvrP2Cp8GJW3x`s{Jw=~FlyzvE@(i_i6;XJ zjT_>~48Vmnh#PbYHd}`ck^EgQ!jAAeWW&<q-EB+wK5&IrX3u zzuSv?<&Pk+tsfCz;B0AVD*EZ~(k6_;)DeJM4YC5%%=AlA=hS_A21C1n2 zNwOjrZHbTR+yYWj?IQGgpOs&Ft|{siTQlIqUeEXmyODX*SXCQG5QpYK-o{7{=^5?x z24__Q_%Zxto7fgBvcM&sPsGm&MQc>$2mG(n`-+>cl?Cg`f_x;;#Lu2vqYwbOwhsSq zbV`f^3{5jN(LZ5?Vvf0Ng&BCe4a za$?x`b*AWGX0vK{&45fZcUO4$TR%sYl4~MHdnB6Iad<{vZkD5n84z|#?BKLwfrGkO zO7>i=F+-xG{S+TyWSTwe;@$$hvMmb*g~Q+9AAyhi9hxZCu;t6JcQ~u~@E%^Yb|diD zk`|*)O>+=?>Xk)OC7ZD8aQzqZSgQcdnAueis3oI^WYI%V-a{{PHt0p0NMP{@QMD5} zoNdLwoHh4GBTJTm#L%-kIVMP#(`p9hZyPNE%7KYw^v9@r7?}Me_`dyl1gs%_uTBDX zjs5Q%;>iJN^5pd)5U$vh6DnjBS?6mH1==+#_lc9WK*{3Wn-!c-U zt&QK9Mu`8?5S|u2cIKj`FU5Ov7!p`u1gHEU}HT0v34Elh~wH6Pv zW&72XHpP0+HLJ--xXigT(^Sud$GBL8h#&oRc_vDNTG6D~dhOv6_;B;maa>V$>-@P) zN8yJpu@s@v?;N9FMh&Py9ZGIZ)ws?ZnZ5dbxCIb_S5Z~Y9ypM9Y@a8RJnWVEV&^pG zWGxDU`90MY0K~j`ITxze#3@ghE(CMS{j}Ui6O6##Y{#g@+ueg33^-uve&Pg!PRMtR z`J8vI?yspk4&gf3`TO^NTjs|X7wS%Q1@!Yznk4X)ogN%iRvC+`={BxBs!;G zvX^bACz?%P`WWY_GPy_`q9Wavv!`MV(jEW!g^B8jep!r8j*`3q>@@WUT4o!77gH5% zJjoe?ISK7WhZz!c92J1SD>sb{Cu5xbVL>Cq?hL+Niu5LhvUFhx#fPU>zw?K-ay}y~ z!b$o!iSdQS1Fj~f6_QJu3(Pk?&v#hkCAx>1Hck_x-hq?}seCG){v7Z^a9hd?aoDM5 z%;lQj6rU$EUVI{n1Y@?P$wp6LiXV$kgTZny^Ij)WZ;#+#f7Pwga)Kb*uYW8Yk$rk1 zQ5hnec}6UBG_naCXtp20>En7{+#vKJ_6&hgyWI!foYLQ|wcVU*we2wbUv-eSA>==~ zc9!@&|I95`e)K4IP0NMX*qaX-@9XzntdLxPwQ{|+XP9-EX%E6M)n|MhDZS1*=x9_> zMy&Vp5r6D&ax6iaESINF;ZHUClX(y5q+J|f(bSYt_4H2*u-JwVnlVYiL&Kl?V&VAD zPVM9s$%AtkOE7;J4mEJz8*;KCF;NFBsg<#xq%$|{8(T$PGE zCyX+D;ND}GnZ!YJqkxHaNegFZjzBAIfuk}@{W3@X`1}yEGXlkin6eW_E{pWVVr}6) zxH$BurlB*T2}?<;!)AC#J0oy3rE-5-h&R@G?9~d!cVTbk{yJ;Sl<+7uu!}vrH=ao- z^Kg9}BIPX%a>)WmaB*CpD@5odGplsqGhX#G->xCDt{3h)5zH4?w?*|?7au^}*X1Aq zzN>@WRUJL(+-jrIiF4%5VbdoLktxH;OSn6oz<GhX!XMS+Q#3Z_?KEF^5wRoe0M92<_q$MunX=j?UZDqgSQD>}fbc(Hg|eQ>0`)#>V8Pn~#VO(_32NyBB-fyHt&fm!go z@jC+Jb+%#s47C7uiYuEe@|0UiBJj6zYzYV#@bv)s$#vmw{X(1=Oc8yca-+CdgOk)TO^Z0 z+dhq^R;=BPYbaE}7vw?zgig0!exN$z&fADzal~-kpkL7>qL$+uG@VB(-`v~Dm4xPi zZ?+W37XwKQOnI~Y-D7XLdiHH)Kn}uvHL5v{W3!^qbz0^AN|9BlIoQKC#od-=ze^Pa z(E_kInMx?$6ts%Wj94y9ozupCc;%1_ZmacVCb%)(^f^yf0C{|XlH3`$ z@;O(<#l5XJY~vo47{sC!;>ADQ!_TH+Hgm*;8r=$?o}Q}RN(Ef+W!+{3p!ZPFje{fr z0(RBWK=e)aC_Dc5r4k|?l1W;&R(R_=(!w+~d?PhnrG-P`;;OpUK!qjz9??mel`3aW z*7aR^xN2M&lN0;`17RMutal}({n&@~QpujF3+H+k2D7BdwcYi4) zE}VxzU;+}z^XV6V<^qct0k4mjT>UC?qkrtyll{n!>t1z1?PlC|jz7w{?8j#iDl&opp>MI-c3vjBZ@b&nN3>=A7!yx$4~w z?2+>eY}(->yCc~u8yY9uDbH{+$2%tAbd93kM&KZmS%YWy zU;Q;0x&m*z8Ol2MLPBqtFH#nvE7k#5Pfk({G^{0cOaSU-mC^LA724}Y`{qYy3F1yoMH`%iY3pRhat)SN6;;ZTFvx}evzU*(Ssv_1t% zVSassJMVR}^-OIP7MomDLEQ}tg^3IzWWCqW9rq$G6{2ngD{cLPUA-E2&0uz880u$X z?%~Y(;hQ{OnPVeUWg$aWZ*pz~Kf#xkkj(Whf^tAw7C()cpH zlj_mGKkc!dxg5I}zMgc3jgU8_2FN%3cr$%9gz*6S8A4CIGiw}oSd)84T$tQ<# z_G>&Qum?Jss+n+p4~)1XIl=3bMl46Y@^5%@N7w1AqkxwC#2x?<(IB%Sv%gl2E7V;3 zh`{8!EG#J|@-(Sau=k%VfImgYVuEUCyX&{@d3EN!T5a`pt-6^6+|5S1LPwrE`mleM zocRBgT(8(X&p##S_ouzGqeF#okHVYW!ULnxyZ=OF{<^>JvHdw|8q;AG7?R5;=}<@0 zkp{S2Q2lJ45OO;|L^cw3WXk22=}a?ZnOR$H+8@mens?_By)tET8w4D%U1jg=EQ)<; z7FCImwO}f=M@8SrAL0inh1vbsuIZ;}9S<2e?g|x=HxXVDTchcU6&JYecdIX^Myf7* z=LR;=QxpSVoPM-D|GiyzeoA$5I7|#Y6at^yk;jP7#Xy= zE~c{6El*UcOaB66P+@o@>BL0(a>xZPY;_{A=kOXwxZhcr>kmV;4}{E?;oETVb%)@p z6;1g|Vs99$N)`)1U|_W{AklXJP@`Dv5Yv!xE5w|Wyx?WXERiMD);!|we-ALKr?36p z|B&eerXOE+yw^tpZi>t;@2#;@!q>B46ygSXF%Ng!5&ssx`H*S3h(dKw(71M_j}0Dx zs0YZ4t62jNdp57{nOQd_`1ypNQP!mm+jz>6QX>+xac?G~SBZg_R*Kcu%?8i*uU%2(3q=1UJ1Tzdun zO48AC3xA2T{`NZqxA8`&OQnO~x*qWu1$AKcupXUbDpd`)F#;RX~@30t; zGO|P0*X7{f^&gA~ovD;b>bDHv-})?^k<+bM3V^kX7_Z0Ftv{BoyXyJW2KnN+Jm+|R zx*a)%vgzQTU#5&2@|YbZj5?S?c{XM~NwNWHzfxA}kjM(+!$!!6WBEw@bOKtnF z1N}tS+3XLoo9j>b3L7g#Hj`Ih)>oYB5$Ro{PVkSzxY)TOExT~Fy|R{nl@cxOe*}xG zh4bnElu~aRw!_@v_?m;I3;9d^r)kD>vg+>kA=SJ_uqL zZC#d#E<2Qcw(*8N%GS~+?=9NM_@nk|i~ox}R4V(X(!KT zIGVLO?IYc)CJ)^23-h?`mO+|+C9W~{9|9Bi1fPZ7sQi9&8tv-ugX=HGT*H#IHV*h1 zTKOC}%JM+0VYqFz*XI>Z@Wf1Pn4m}I+URBVUov6f2Ca+8XuDJAar8I}93kp#HE^s3 zA*a4G*M|p<$Scd8#=ja#P!vV}6`<8D-8esDrE@FPUQFv=NiaH=%14JNx#LZGmL2
i0Ep(RK4o`+Ld8I#q$Wa4$W2Oq}4dKU+ym|*>N&_}1M$hH?hf=*llcwEj=}jUBa;LsuKjHm8yHCPM zZFv7w-UQIFMrib%y!_>?2+U6G1j^XBC`PJQ8w-S!&NyRtNh!ARK5E!lemfQwqltoK zJeECEc`sZsXB%rrwbqBhNbQ=mmER{F4&;KrdS;YA|3Naj418%^682TriLt(?nAxAr z7?|U;!0dpy{dkI*eXQoA+tedAPZ2Wzo@|doUSenjZxCX>xOg!y6g~DOg_xf7CS4b^#(?O>);)0i+k6 zS$mC*=*7@}-@ZR$vxS_Z>{oTlzjdu)_+V`({4|aAUzxj;!XhX1dx>_%A1uPZS&&Qx zq#h8S2AUGEVwr_Goc)pl@}{F1nY>`t0&rr^ethJ@)H7HQ@$KG#&QbKkNlMv}Gbd3^ zS}O-Y%WTW8Ys>t>Pd!z?+^D;trsy-#GlmuLl^*ye0$^aCh-puTr=x0pJpI`zuG}qq z{Ap|8&or#WG)~8^XRaF?8|v#d=X6mXd%>8bh%jyQ@8#w)Cm(f#wVp;eHs>*gO@)9h z6DBHSLRyy>;-Ty1m}Ja2DLi@VGS25#!WaOnf2>$}2F39}D>g2O(@jw+(+tT8AHTD? z*GCocx!$noX^|>RffIgyd6%+71a*xLKp7UNXCWnw>j7`Oy>{j-dTVR% z>Nue0_ly{Rn!gS1w{Fjbo5?3;^~-Uw8@!C70kic^2jvoeF`1Y?AKEDE{G10U0#oq< z-5U3c)Bqr1GBhgjK(n5C$2VwHa{oSoU?{UOySGI>%xptXKWKR9Lfzdr5D4-$cQ+F( z*qd?rt1?_a%<`^z8F+z~mf`(6dp^7TH$rsFCO7vfCGFe)Bv5qiTJfDx*VE~i-L|y9 zR72^Pn`tVY*^n`L>{+etzZ^ZhSAYckhr&BS{9MIaXX_EZDWwXbzozXKnjs1{v&Cc0 zfQ(Ft)#d&XKR9{&DJv_&zNnhWB=J`&^M?vf$4AV5TV&H#A{Fy{=4B8M;~5NPw%TCX z_bL2XK|I$is=2_}J&b1As~ubXHobbA81qu*#z&Y7-zGM;shU@xbodKLPhQDB zH%(!*ml_&;tY$Mkvv2r$WtZ*Afy=+tRc?ZDY5`e7jcI@X>l%jg&@OEd#wx)_{*!7x zh3C<_+#zv$?C4u-n%kgB0;L}&bsfvYm-v&Dk&mPZVic%del71L1`epOPP^gUvuBv& zb(mE&Z*@+T-i{kdBB77j^|vE=^rY1-FZQZfx)?c4C|#$PBijAN?=|-pAC5xu>iZWl z!EzCdicSM=`3!yJE)!?L;LE(xSY&{+6$XpafIH>(fB9>Cin@foA4O;1W}h+{P|s49^JGpoc$dR z*Na`VSN5Ztym85oWH{Tr@*KL(9soXHH`K+;o__KF>3D_LFL*m2FbF>hI<3i-bkzNt z2%@w}3C};b${>~Hw(NPdz^0dM#l5>=k&2gAwvlKT`HYjV{k#;b8Fgj3&R7;&T^bu= z5k>lSa$O$raHH|%knuFvM&dzUdUdvzC3O>zO8zpnj$H|rPq$0%!zF&s8nScGj)BAxFS-d|lIM=-iK6#?9t!FYAxyL*^~dqQ-kePH%*rt41da6qrIg}SzYqe zIe48UIBdH}^Ch-Q)+wo#I_kO8iu!M$4;m)B9Jn zI@c;R1GKIG8~na!dV4kt=wi}_{_$Dy(~JTBRMH+L`k{_R>^aW&oCpGfjS-KkJ|WLX z0K*s2ZnbNYD%VuvB(3lA9v~Y1E$;iBBhyZ^dZz#l*27h>fAc0JZeS45adzs3EHrm; z%44Atb%PDk7uaJWUI%t9B%|xQRb0?cw^P;Ju3Dx%9}e|N)AsY-T($vMS&$vX)gkFY zV$MkRWvFxnL#O<87he(yD8ur02?3sW9YV`>>Yz)Bt6`?u=|W7tvVk39$UuYyd3$Bs zatVOGBK6d0QJV{?;I}$747lyR-N~km?ejd3d!elVWrfs3$w%)uI;kM*&i-x(K?+@Y ziG-f-KqslC((%m1aMu_!2fQqhz*ImUBJ<*g6Q2$B6zeezG`7vTvBOxBsSB(dP1x)iU_4=BF1k68#CLf zHEwXDLV9hDnP=R}AX}Bs-ow=EK42W~gJ#ZbC$TGVDqlki?nI|9T7CAtquM)ES`PC8 zUt^NbObaO=up}yFPeW8B7s)8qS{<0cQD|1q0DWyzWp3sjoSRW3pQ`HE6|wPDDn36w zQI~ob1ATqSN2(R>yx!E%^vPt2+@Szz_>DYEt@u*N>mbX?Y9&<~@8!}${Uk0i>Y{HF z*OhH?{0-O)^_H2y+`y&r;si@8XbWo-uCKqA`$Sxlnk3UVt<3ePCz<$&Idh|zI}ve~ zmx>Hj{$i^>)raHbLBU7iDPqm9S@BaLbOEvFueA8yOk!o%;F%3h5*Ut9h5G#zQrnY` z*h*D$)V(%=n1H}vouECU`v6I1r}Jsd=w0XC*=fq{_TBaA6%+$HF7{^r< zX4&NRSU^L&di0Fdr)kC(GFaxLP$MQPwt=fRM-FNEBM;u+gB%1D9UEgO!zcPaR=~-o z)PBp3+We;^JUt6ii1RTNT^VI-7N~O;8|!%aiTLB0_VTItgWJ%}e#yP~xC!N$rF)f9 za|sPlzNZH>axr=jaJGx~_G^x8OFJ->ipKrfJqM7i4UwMKUav_vr4oWSIr-z6B=xza zUC!P}drKA&Q$|oPOhY37i(mhT)xA{}y4M=sW-tZO-~Gca1rRyAeU=2M_49s~Zp@wf zh0~TZJRHqle6~o)>|3)?vCyHAIP1;wZ1rPPy9(WJ7vDT~xAeu{%1DQ`Hqke~^ydt6 zZ~-}ASHWpl%N#eQ@cQ9e)BlgHvkHnUYP)oBf+rB%-QBH`V8H_fcXy}p4i-pocbWtX z?kicS@YVJpUw&m{y?tnKHa{HQy6ISeF>7Cg#Ci zgCjd+vLk6RE!g6xCkXx3VK0+^RWN_mANfJO2%sOquraOn5W$df5QlI~UCLry{fGl~ zJsGRvzoP_0m>%gmZl*W88u!emEP{`tV-K|N@j>#P{wyD8FR;esRL8MEHH8;0DBDG3 zyt8|P4?y%$=5bJS^bkrI`1R=ZX=&}5;`OQ#0yBPPaE6=%gUjne5{QU;hVOcYaBAp7UG^QZ@ z+b*XNM7S!%TY<~38g;!rCbsZq%_E|*8A~ogukK6gA3J%PxnNZGiFOn1i!Qp?Wz3BG zGI8@&C~Vrc!?k4B?J>=HY^#H*Bs>rPTomkCcqLq;FxEu(f$eA@=;yGwIPR;y;3NOG z=>C}0VxV?I&+0h?w)kKk=m@*z{_6LSmZfKVism0n!@wUS&%_YCP-O3Q9tIVXXoZiu zKEj>rbmOtyW3-l&?^4Fr2*vWcyKEnQ^FYF@K0?T*Ry*%$VcJGhv1m6`=fOMgp{^7X z0mlhCR@^3k<}q`>0uSUkNvngEB}ts0yq5QwDCx0bXxYUk{CapPXMWK^M0KuZ^#g-(C;EPw403PY)hI0inj* zK-N;D%D{hpu<&j(1%=WL&Dgg?$|0dt%PB)7&I&~UTV z4>StY$J=$_0W*&50sD2KdV5V+<|=e+Z>WyjsjuzWEl32By&KZHR>Mj@f~x*jZ;@<6 zGcabW?KhMYi&EMhnQbd{zi@F_>961X{|=QH%>NhS+ycRE>c5C{ID|#p!-igIE1r@} zvE`0&b1Vldm%PuJ9my!49)8^8R`9zRlCbp_n_}pNS;S}3)>4H=Q$aq2Vy48dZf{zl z3B>U?KRtb-=C!w`6^sf2gndI`nU<&<<%Ts4?7)T=#d(k6R-<@+;W<>o&=|c!aN2<0 z1n+#z*cUaq^G(!}idHm;sH7LV_@2DHa@-%7RSp+>pSagnr~NJsU9N@@W%53{X2ssK zpM6$KEnGuBkpA8=-uft^1Wk(*&Jt2Kz0oYV>Y|ZkONgke6rUS4G<^f0gMTSo^P&PID+t!j(QrP{Pf2J@ld{{$iK>ps`;nZxJke z!e|-OL;GpUng)POM}A1u$Y->}F-*Yv+$c=2DRPl4i%F%G^>c@IFPbP?M8NMs$Nb{X zfFS26axNU&hEg|mAQ~>^N<~d%_;7qxFU-Gb(wdY|5?#S);9q~{(aoOlBwv1h4-TgOn)vQL zZGPM;|3HIm_p3vTKw%^(%2v{4$*HQZfv9?ml=5gN(SQ7&7W@IDxq{K00 z>bU4L`*3PT1);~FebK@)`Q|u~MT`(*%i0%Lw44 z>Ckic@b)E*3H^pt0RwCILtn#vn?@X*@lDI4Xy`6B^Ae>b+a#UsAy_0UT)4VQ! z(V&C$q8FS4i(plK4z$Y2c}Ctm9c!)s-aWXh!0HzztAe`JyYB-hxwS(FagwZY)ud87 zD4LWO6JE#zU3F@CrYwI8B|`D-n0;cLmSS&U!@-Z~9Vn0F`+m7+LI3Ar9q~#6!08+8 zDZiHW-&cFzQJNTM5=z40f17RKasmb_*Gci4NWY1YI0k+8sPD97bQRTdd5`$qpoV`B z|B;M^L??^8lzQL1sv!}*OfQsx>KrMG;pV3{ABWQ7SBre+!|pI~ocrKxBFWD(P~m~< z!^at2g}6D>Lp)s^%PNydclPRYAbxHiZNtDb0c`YT!<+5J?N5|`F`Dqicb8k zHwIIhA`C(jhz**m^a)B84>CD*6S3vQM*}*mww^Y#J3eiK^QAM-xLWOR$EmLr^I9C$ zPYOZFhiY4ksH~I;2T>W)KL>&yR|YUQ{v88~MvSmzi(<_BMIi4R#1kNfiAXZid2FV0 z=ef0=1vgRAO5@>@aE2gEJd0^kz0Y|+YS*Ik+Fr~1p-cpZOwgBaBiN^VqN~z8kC^|>8Ara_(;(=G+Zyxt4!fzzh3WtQ|Ok+x1!3*7XAS95zlV*uU>u) zJlm1wTLlwszQDSTUq1Y~D-`EWf3IAfU;|F)tKYLF_c*yK6AhExosH={TjUJeoc~HQ58zr)PWvx)g6s5lt%QyckQI zd*7s7w!v^r+B&+7dG*vGJ%b=8q#6nA`P-F0ND%l)#yQc16aJ?w>ZBz`-V+b{MdF{j z?7qb@m|AGT$kLovX0FojfJ83wl^f1%`08HwYf9^8?X<1t3zqQ)W@&aaPO@49w^POv zkzFrGu&8W!=%F)vAd@RAWlvud>ds4&8$7NmG*Hw){@+ehq5he3g$D zhhiL2SbplPSzOSRhLxwjIuI8AvOq^w7@7{oj1FzP)At1RUV>~mbxu2YoS=Jw=)R2v zpG$t)g{$X_vu!u*V=tKnq6*zyh>kzUy_EUD!CfvsOMs?Kv*t`bkcD$Q<^e78*wqw8Y15zlC-U)8N7?p{4V2@v44lX!r_#k)GJFUX`_U99NKoX z|6Hi3hx3#Fo?2roLBZfn)rDC2%Yi~Fc<~-<6~9O?&a+(+dzCtMq93z;{Z@(bqD>Xa zTGG5fcCk)6tInOquI<@pr#S&wW!&=o^EGO|oq8KVPpzly-+UHaVn34>i84QYpD#=C zJ08UgI9YOZ66DIL&VnxqECF>$hk_MomrBLL9|rN8dwwm@PNtMeNvF*u6BhA=)nw8v zFs@9ftd6F%wDG&G*bJ5~VmI>zG+Wki0G+>~Hb0-i_M+);^+;q)_pKxX*qRS|iMN9G*$I=Q4XIa??6O z(XRz+v@tZ$N#9ZA7!G+UOm>u83u{lWs{6&8{6fpbX7x5);U#uZv_^O*#jFyOmGNff zZgeObKe?YMH_*9~d7C9~MV8Fg(~qHuyBDjNNxt~2vif*!)b?LhmuLMiQmnyf9it4vSpHui^Aa!qvY7jDGj;;-a_nD>5comaH;)P>1N>YTvd; ztLdZ<2kWanw%hj`wsX7fhj&5cKBsEezN(@?z2{}ef!9!M zy{G?fZ;^8hZP>1giDXGibVT<*eplt@2xUs*-v1rOtE*!>VrM`?0L+*D;Y)Y(voGaX z$Nr;3SNyDwB3$%1<2Wi(I2vdvf!DF@Nnmi+=`Ifgj>*IigpLHiZcFp~i_ut1NpT5A zTVfFOna=8>l|q*z2$?5)isvs?U9c}Drq1SXQG6wZA-wS~bxUk+gx-=uJ;a@15(4s^ z&t#Z&fmg8ot~HC_{(e9?&l1$;cuudeF`j!xJ>hNnDf0I;Baf z1v5gF3XG;pBNU|_Lcn(fVR7SBa0%B{>|@v^{eAtwTl1)?@4eEtITXEPLK59d(VrDe zxs1xoSlAbwwCzvm`Q1W4{qARZNpyBtpAjs=H9pto+)e!k%JkL-pU^WTDj=mYmpsrd zo!Gq}#CxWys#B@3KgM&s07mH>^G08k!oIs8<-z5NA4LX6qJ_`CI#C^NHp53RO`iWl z0VSb?@*4%S&To+g_(s4t%pkBGgVZh|LWD$H7UyBje@ER`;x}G>B~Uf{$<5TR(ZbQZ zbD4ulh%$K!bNM+Ew6`$Zr}y_~fxj|x%Nl1daB#P#NaCOcgT4B=B@c*~_iJSFPZQIc z*QdF=G3{~k?G^(fJqBtGJWnVs67!@46)_*Xr6ldKYjUKFI`<>W1g3Q9A1ClG#cPE^ z$$ykO4l6h z885{AUrFx5yZqO_V5TdX92m}e9IHMQAC9$Y@S=04CkG~iUk)Ds0Iu$B2;)wMuYURD z7XY$3j!)*nR*h$P$IpJ`Hop!*G zZ=*0z;sLkyRD~2fV2+O{-mkE%!v>yTP$?Fn;?QnC>gIL0j=Kf}?qIopYZevuaioJ`FDG+k?&lZYPoXD5S6Y#i+cLP5CWyefwN|E;*L3dy+16R| zfR$QP_sVZ=huD=$rOrm+j+@_plclizL612IyW-x@U{U4s z>#Ots&IH3^CMYnPp|1^0zG82Nv;l1}D$hMz<9SwlgCZ^>a z6`yH={AKiRMm!IKUiBt;j4MMKIC9dAu2~2Q!aJ&%l?{cN4By&HUwFo{GtMUBYLACX zCm&n+WvY=GhKVu-QA7imWJQmwWSUq&7u|KYORD+qMt*uNw?HR_Oo|n~t)JZ!!(?4k zp=(1=xweW)ORKo|{^>rq1GW>%?o;Iy9Rg&_Kt`?z(19wL%z@|>^eFMTYrGN`doOvA zHJ6=Yl_f-XvRWfh4x4?fds7Z#|5P#*7Gm=`@E40M+G19X3wF4@p05SHv1~4Fw1RI~$K9u(3G*8RaOHpMs<%9^f;_?! zCA|8Jj)bvQC?2e(Kh}Y*Zl3xxtE^fWt;+>Hff~VP7*q-u->z=_W2BI5wyut&8@wqx zd40BlkcqruHRYySBy(N0_a*om!>zM{`QK+TS6b%Y`%P6=cy*DJzY8SjN-;FbDhp1! z##(YA(@v=jWm91xc5#X#lScQMXv42D#ZSdlG0dM)@PB&6uMRs%215V5so*p>wC|N$ z)jrXd96G93Ub?4D?PDJa{LLdn9(K}|Sh6AfXoATBfs>n;ug^HwPdU`N1NJ%J__&JP zf7zSvZ;YO@vvxwEp-f3wY1g7>u33v>ee?;+N#5M0wMkbzk4Qsi^s0fzz(_{cI-=XQIu?M0mk=HW@~e6CHt_hak|A^6W_qjZNo zFMi#7RWr`%WwsX73@eguQi%nG&ze7fY!$?JYn6Cy-s=qJZ=(k_*UVvHR(y@9e!>(y z;3aMlOW1e$DMC&1P8O5n0;xXor*L>60nvw}aje7oU2hpD!oTQUZ>RLOoaC$SfCWAe zqn@r4A+zuzS>7tzKyo%I`1hB3s?4|_=kA?SgkBVL#PYwg(ki_Ok^6HisNcos4Fe}G z=nL!U5Jj2^^6)tSGR~eP1y@a(%HZorzG=-bEpThQa=28sUbJt_EdgsxgX@1+*}k~Z zBFnJ}eMZRZ|4^uB_MJtX@$h6|nvCb#FbAAfa;h4WyXi+5&$WgaK$Y!vI|-g1h%2Zc ze%Xj-y-_3^w=~?j`-Z@)%6DA;tU<)ARZ;md1`T%iR2%^6rhqkP??LaMgS;0GM&3Kl zJe!IQ<=Xt6&Hn(yqbL*2>QrV^mA{%`_3n(yy+(ug~D$3`B9 z9FDnk%R%LtID7P#%0ENZ6Xi7_&FDn0;O70&ce}HEBAxe*7eqqAqGhos={Mk~HKEyvGNae5kjbYePf%CE3yk%73*F-}cS(H9%4mM`aM z8YmpBL#CABH-&bt}cSUc6t<)-sw{?k&9;P2Vtgwo^3vQ^AVX zbjY=k`_eU9dp6Ihxnj(uS5V*`lDCPq&7#Yi)qt`0!|`=-qdy&rsPq8Q8t{woooD3K zEYxeiw42UZh#{NTf-^WZw#cbz9a|otB=OFJL|l=&4>(59zDGUJ%_?@Wu5sWX%InVE ze=rBTZ>R_)6gHFhrMf3nvfPxWu=M5MyZAcx|2+}&E+kl!c900zGh8mZ2%6r%D7fn< z{l(}0tj(ST?f!X$Eo6XMT|k&fvU_07@2Xeb8H|*RWc3$8A@Xdyo|_=`pLwJbtwcLB zMv6x-GUSrAh4es17oE0wr6E*GZ|MMD)TJ=CJ{s|JLu=4S+{1bN!iQ}KVHdtZM zsd^HJ*z)``ela7 zr2IX>ecrs?6;}v`Z%G{y-i%}7%i+r=<*qpi8`o-X4~rYmO2;I+5gpFfw=IMkpUlN| zrefh~KKVt06%#N)cPs|?-^^`6ms#Sk>7C-HHRo6+gv;xZWVZOI+^%L(ky9ulP`axJ z)5K*P8jK76(fKv2pp{B)dD(uI3160rV!U6Ziz>WT^Y11DvZ0wi@KVJyz9n5oBTLOR zJt~cO7@MO}r)rRk9SXI1ZvnPser-%jD#QhqR7C@}^yipMyE;7m#>PXr#1*hq-~K6tnK1K;H)g_%sr!%84{ZD?33sET~?kdiL2?KUu&pC6(WTEuK&%^tbcP|Sn!ugZh2c8f`W+nsnT6wN9 zrUHm`q61`L#{a z$_6+b0B*>RSax(LRF!3g+;S~=al@15FSMG2e4kS==Vux?vhR8#!!m=zu4S}GRal%2 zQh&q{XW)03SY_~nq!!!EM7)T_*dfEsgpp&l{Xap^!xIOiv0~E=BPdl>5G2VYnsMa- zb61N)FH(DvqpV`vsUH2wtwyKNSk*6MPZv}5f~r)@$qmmfP@H?x6nq15^cHA=W9*qy ztih9D2d<7tQ5H8G)Rla&Rl@ILU|VKusx8ur{?ujb^tUc5w^9hj>EkwceX%y!1!n}} zVU^+ahzwWW{ z#+C=3Pf7X*wn9<6g%PoUxsvbY@1jY)y1j9jG%>@}2Y=S7!cR++!pE}5yiSUGy9{iE z;TK{IWa2J@Y!Fk&#hYHIc#lbi?X=>&N&l#!4bx*e*fcLg9vd8J=ZP0E*1Wq7_wOu&IYIwdAgVLN@j}jtG9mkGyBLo zb}VZQ@6GkVq1>op>3R=p0#{WXO3ja0vW)&Q;>Z4Jk z#9Lh$Npg0HBuIT!OKI+HH@EJw1Q5%8x83I~B$Dm6Lwdr> z1{CJ$Tu?SH;#cZX+1d~sd_Gkb%OZb#9_)R0NarjrGScB3iZAQ+nwb2X%6X z{B>B$LND~X?lB$a3qi*I0`z|lx^ANotHmq!7m979RQ>)d(Bv9%KeXe00QR-m_?^TW z!Wb9!y_9^pR)=6;9|F6eOZBgzqK4;A8MZR+$T2Xl+yXv*)1y+&H=F0rQZZNd@wD57 z{o84r8gj1cRyiIxzlAeGEhI6gm_+eI>fZIr{C1;?h!p5>NksTbx0>QDv4t)~#ei?! z^j8|(yBXtAD&u%k=or7aKqlBA!9<)v!ccKs`D_8|S(_1&DW|X#L1WZ`r5qIgf|>Kd z@)u^piHMNJpGKG+ppFEf#tT0Y(B50=Z5rE8vi*yqMO166H>JF0C5Yp=vYrJI@5NFJ z34vwtK6^_`L?*zW*tl+i5}I%=#7lg~gnrF~%4>w6+s&C*c9d2$+TY{_Si66D zSgVVE*&D$B$b`alEfQrmpN)b_W<>uOs_-K7PfAAfh-2JEebFNP4;ClJWPogZE=}>3 zMu=dx*uT7|?+wfsK^dHfN$9Q{(7z7Z3^lL69)61(ZLR#-1u7&M)$x5ZFVLr9Qu?IZ zK(LGS?aI8zsZr%?)P#M_JB~0{+FBk_J%r>z z&K>&K`DW0+{WSP8&F=<*QH9^n{p3J%yC~c{hJM=$Z@Yvag&Xd?X7E`@pbA@L;y2G`Oo&AYu&^dU^` zM}bs^cITyoGluAu-+7&@w=5q|62&2g9pdlVT=>U~)Lg}w+zMIvyhACqpU8h~#PLC%x-f zCc`WKtUt^@q_c3SiC}S$yIA|~$$*DvT2O(DXXM*fh$zr*N7>EiBRkq|YJ-SOc79C^ ze{H5YZA;GX-3tz;&Qng#i^d1}oc$8vE2g2s1%3KpC1ZC7kEkJ3g~~GhqDSd0 zfg8`fPi?k>r!FA9(9U@)#1QXc%XEALei5#JDF}I}%cN50NkQt!ee{rCyxveba{RX4{1h=W3eKV~66@eY$yp{l=Abk?WLPL92K*>xkU< z_{bxrx;zaN4}1?-hJA$BcJ2{~MGF4yKo6OX!FP%DGIxQ-^}_$N%Z zcwDIJo_{|fhch1WS-rmQGk%iQ2e+L%R|ZiG#e=I%Kz=P_v7?rpac6|>*N)cq0y8Ny zBI8Bwlar|&N?_1kw67ck&;Yahq7*|uJ2;OYa?8-jfGkH7QSG>*5*@;WqxOUmqwaLO zl^E-w(TW`m4x0Em|3Mbupx40{ohA6yC*{EUv^(2%ci^BD^^eD4B?_SNPpRP{CFC5{ z{M7fU+s~P~bZM7i>#y?dOMK6oLXi=lKmeThO2@4&Y5FsK+mV~aVn?91o=2YpqAXZ` z?LJcE1E*Z>Qi!;}F0)!w(2 zPlRf=`Jv9=J2!ju1}I)^?x?*cDq1dGB^ay{j!iz{y%BI%a2Lqq*e8utWG-6myzuQY zb&Js@yPt9G>Zxz7Nk}0_^*hGu#F(}zOt9+C#Fr3y{56p0LS7r@iuH%gvV|PF=QeEb z&>5cnmBSqcYF2qgbhe`MvglX_BxY#S7P+B9n6TcgRC?=N6`B}cQO){1rYfWY0Ujr)HXW~xnH?&Q-XEb1iO#Q6vtyzaskRKkyCWWR496S9tL8kT zS!HB(A}+5yo;|Knt{v-p|6)8r0^_M*=tAdQy-Ios)>UHuC1vgs-8*nr@KOX z@M?k=xo!4%_+})`GOxd=RyHOKHzxn-_EbgSWHpEs-;LO|fBg5sqw}X0NBkMY zQgb=1Fv%upE0eWP*Ah<`tP`Cv)$#f zjopd4Xp*m+X|a|ijT1C%4jg~?p;9cwC?} z8%CNjNGp1Yh6RTVdf0|c^kKcrtThkE-E-%1RYx91Q``T7)Zn05k;1uePWZ*s&jRu^ z!P;G~=SH2(hB9d^6WM8v9^QK?$M1+Pf1qSk_;v8TjSNu2^7r^3TFK+*0^p%q@ z7DyS`$~2V!nN#p6U@`k$G!j1Z)EqdUWaUCu&VGmIl<1iY`KbOFv@lTHLN`u+9lkua zJ`0aiJHi&`!XNF1SUGfLh$TZ1sMtnTG@>hl*>Y3Q zw-{Fnn3WpubjouXiv~k$A8Dw`pQW-Xi&6Y*>X8gSOZBZ8*|} zzl+07L_$>dwhgAh^Kfu)Xe82xjvHY2O!`_R|5_9d{x`qhac{(I6!xAu@=QI1 z7ULes#@T-OJlCIfMu)10E=o)57FMW3E@_5?S8J)JY5xS4SHa_&&Z6;vyCzfrPd?7o z34*Kokv*LK6}*?qZ1qWVw2`y07bj_c=vgXZ=dBu6)kk>}9B^n0$yzc$?| zg_~x;R-k`;MQX-|Ig1j0c~n?Ou=mW9%?Lfeu~smflP8Y9TQSt%-yz7Qfzg`TAqN!5 zGU@Po#VvaBU$DYBbq0P6>fIU46M41fEazAuFi$wT??Odk?%cHa-v*X*{+5aA^^(lT~bS_$-sK9WwN#t(z*=3dBQ1jC0D;5c;g0D!b- znWyF%etjT6(`@8+MYH556{>*gu@tR=auoPW1LK(|Es^2JUmX3)FtN>B6hl6cck4#M z*9xG1!(6z2WI#c`HP0BCs=@qDMePy{CXQ5p59`+C+{KH3<$wE<>Vwj@ET`>z^WN<5g}1q zfJ^8+3es@Vm7&dBM7U_K#b zeQu_58Iy>E)&tx_YfewTRnT2Iq*)FIpg5By4%SMr;jKYF6>Jx7PAXq8tF1dW=Q){YCBQru?EB}cCkUFjN zZYBHIIxdh=1sc<31&EQ1Gtb2nX)(G6am=b0KTzp$HXKztc_p`)743J4ajm7!`{tay zPCEN*w5}0WWrcJ;Igx&9y{%xF!bRP$Kxy0Ds~$oRvN^rcL zT5}2vYoLFywlhGg`Q7!R$3*6s z=K9ra>~bc(nKe)BWcw-5Iez(*K!pOv=?K)d<)~U8INkf9$$8GCLo}Gu`_3;?(goQ7 z?7v>@F&Md9Kf^@VxAQ#5q4jRR_N+xxB*^$GRlBHfo+)r>ZH0vyIlSY3HFGNBea|Ed zjsqC;mHY34ZPaC}*R}pNIY68+e~(t^&W8tY`1R~=`7hZ}7?WYGKZsrZ;SoK5(~{;w?yq(_h_a%CvHpD4B<07ry&sLnnS2_bJ~V=19fBc}-AoKbr;`GO>( zwI_YPf+g_OUu?c}Wp$)?Wv2SMgD}WH2i4^cqRW{gl9i5iTKmB0Kq8t2hTcq#vx^cwLk)XmiS=%lYlwmg*Q%LW96o}5%(8V#07^sUvciY zMZ#lg#aZ2mPr)F+BW>Q*eE@Q~@pDcj-R%$T#SnhEt?;VwDxuiaCS1BT?5G?@u?x~_ zb6)s(u#UHiFj>$5cRMx(;hE8u%j4wKj5?x5IEL!0;A@nJUdZi+SnFga1&C72^RKYt zOQUsGMxeznF-PsemvCkKDGIS>VQuB`;aE0NmJt#sRb520xs9F!YjQuzl||`yWdyA$ zSQObXlhaL{wGT#P@1r5a$nw@vgo&$yuCl&;pGDdDyj^@z90IyyEl_8XdSvrsdm0(A z$;h>WNwAMqoppRv^(c6JF!ebw^7rPfa}x0V#_LCQlVSEpi3d41v6`QvmrMJV6)4hx zzIprwB3_QFfFLij^DuQ}pI4h%+XF2d_jE5(;j~q>mexBj?2eot;-|AHL#b=Yr_$?b z2ZmDw`H?t(ol~9G>SRi&n&lMd8@jL)3 zatJQM*TvUfpZ~A`=y5+20dT6-%n4iF)I-rpKG)I@{y;)M?~u?mM&%OMZ=-?$xX#A$ z^VB#3bNQeXUXz~&a3s|z)%w1ga}{k{ik}(0e{J?)gC`2IbI^@}Hw>QQX<}dy((2c! ze3r>%OHQJq@1ab9s8_M`M*Ojtp?}`?Tv$6Dn_N2=0i7oYE#1Pfc9J2AFS-H)%WK5A zGP=4(ZmrRgd7K~Jl|h-W@}@}*xeQX@sQI=(17@DFYJRN#N^E9E66qTx&&}_8S*gkU zwuKq^9w0L8#yqj&K1tvfjCWTU>)bjvj&Tdhg>x6@PSC)yXw9B~Xm6orz5gw9qxUOp zZSbj8@A(=gA>EhpK(qBnTKzpsx!pUVX|Xcrsy#9u8}Cnydt&aqS0S*W%&o#rK8;MTR)KA)N9C>LlR-@M_gGgFw$tGzQmVr{K7|8VS?9SNPl~r9qkV*A z5p>l3lcxqSmJ#PvrA~fYg@_!pK6KYEPVKob7CC?|kM(Q5ui&_`f zLTc)YR1xQCiSPMZ?;lVlAN}(?e@Lx|;%j2@Vs)iKowY~`H3;PuhBP~Wt(@LTgc(Sn zcN0@VvocQnb(8XuYsW6f=WK%NN@E&qg1K@s1PKyHFcbM!|4^#eD=Ij9?S9Bw`Rr%d z=3PJ#RZ@yx6-)48+!LRbd(f>@(0L?5jX-dUupnW&PciT|{HagOaQ>hnQ*fHVjSiIn zD#WNM*^1nRI>FHqyZFo0Wmljd$0~(j)j`@hSe~d+X8tST1efAm_!9y#mt5r9Yv^aY zj^LE<)V$?2@Vue*{c3^yQE7R{s<_A0(k+tArN0Y^`M5ez4pkTQ7WLn2x*xH*+Kqm? zZKfTSGMs^v{j!vFzmobs&NECzEqpZ@@bi81F0kfr*vF&-g&$dmYe!cDg#B@q{HJ9$ z?LX5;AmK+(O&Tqp%md;E%#LNqNFrR1Vw9YZ{^zbA*JAxc6^#sNwXfBsZ8C-yO2Smg z;8uwBH-y5SiiY1;d-mkr>W0^j)`k}eEVnAOq-+ps!^o6<5rdSF9gL#0omBzNB5bdV z#3!^4gK2IhVyvJE)tuye=y%i*DH>Nz)BK$|G5m!bN?@7eMIQMbm0Hi1{1V;iuBl!K znY+@m5Y$i-gZqL|W3d`uu8TWBJD_I%I3xBQ7lJg}$B7^Ar8Jz1U{20)jHq3cwz5X_ z5JrE<>e1^dF=8|@|EX#o;=C8c>+`$(it`-W3xpdLUFv@a6wAJ`YfC-5QM}~p^-o=^ z&L4qc$#-CT3H$pc^eLWyst5pi)*k6xCt732Xj$UyoCl7oDg>D#Pk2$k^B=V*C@Kxy z@G04|;6jvI!BX!d=_lDy_i@D}@VhZW9jsZQUA4~z0D60|F;9 zN0s)v3!yGar{;n+4TgH4J*$(3Ea1Sf=&1QHmc^Dzn_PmVdZCerTPhhaw4%c@(W~=h z$yLy?=d`(*VDw+m#ESm@TEKZF5`B0`PVpSqm&GH{8U907SM$ZZ>(k&GMCxuU;C0mp zGuaG7~Msx0uFj>OTr>96E&5O_IaJZSE5 zz=B_R1@bPAApul9w>+&N~DDWWW_=e5@x^E$F&u2JPGr*be;V zIuNj8RRTa}bH=^L>X3vO2XhxzT7iXov6)Ff*3lnSmHQsnQ5^yT8nfRjG?}xd7v5l( z@B$D2n(>FELEA8oOlgN7HK*MzLvHBR93YMZ;gFz~vGH}Id@O;2mlh$`eM^o7x9tJn5z;kRY|qnoM9 zg1U8}=+46{iVkIf=F)-b%CMRBQ47CFcq2uH7o%#AlR5qEfY-4J+msu2PKxJ?{~7`m ziqN5kgN|>2Gx-MrxOI;HX_uPykIS5jsO@TRWDFed^4W+4@dMq*k9FcwI>hWZ(?PGF z-z{#+OZKkEsv*l-1qQ9$2xaP92tY%>>HoBS!Q#Ndb3!M}=MlqqFk5I_?pv#N;r|X{ zdHoIA$HmQh5^Ln(J}qMGj}eHL)3kHtjpMCLMSMIL7-{I&1B788i=zY5(%JsdLGPt8 zpg$9O7;9pD>cIhDu^5x0v9xzKtZ(#P=H-hJqd|n$=-Z!GX%meX>57;?8)}hlTE^xf zwQbC7YH}GBj6!Y6D|Z|10$l+7k0Xzzo`L^%TQIdan|KHHsbBsfe0#C-XvizJtGTF0 zYit=ev!B8VRd9Nz#piTr;Q16j8rZJ@s7N@M4lsvd9`#n%_fU@nu^~+1a4w2@5sU?n zLaHaorpJkHLSy)(zQ$j5W4Q8l{((Qz%ja-UJcwcK6-Fyk&8Oiu#Upn63`G=(uXcE{ za+*2^nNO82I-ata5B4C>dMxBpA}{Y<~1+RtY~ z!wgMs?o3r!QKT%rrZYgdDZo| z>Ffi=e7=XR;``OoYoA@&`vVZR)0jhk7i61yc0R`Fsrdr>qj6pctETq6i?wk86p9Aqrj6y;!$Gr5v;G&;abN|jhdZaTO4{W zpY~SgC00kWeT>!Ht@9-8I#v;WXzm0G$MjEhpc`>2wod$@Vf2|YZbbHL>bRIMC-E5O ziY?oX8DV2N;cYZ>%y(?x7u{~0-?6euYNytiYWC$C$~LUw3wkA>`Q}^`(1$h7at+Gu{Lq z$NRKoPg`LSf76Umzikqbr}g47bfY1$3pi$U5py-gPpbTMgS{NDaO+ULWyvtwXB**t zclG(7*`_qwhzIZE7o*cc(7_YC#!RZ~wqKh_bG~VgoFvbMyA5v4`p%P2(Wb?_LMOow z_hDC*Ypug~R<&ZS{Q6J*>-Xy)0u7$p1@+R-=E#1!`mqOBCSKr-PTc>Vz5`Ib6GB;t51Y0u|+L%^z*p8ECRo>&w3?f8nO@!N zD9zF-16?wvTh|m;2E&q@1Z;}ETcb4X__Pw3I_%=|SfzdY4$R&;@959v5Pvi6`hRNC zwYHREOZ%*QMZAyqFIf3|T{%%xJ;Oay_z+E}BGzDhQv~jEp&@Iiv zX7x^4x6olUqL@R9k-dE+#I*Dx9KzHpf6DPIg4RzB$FP_m6?H|m9Oeah>P~w2$^Y#J zEv@ke$hfXUDfX2E<^`EsoT($R{ML6qzNxRS9^HtX_pB_!y!q&j2|>=EkSCN**xo!; zK_UO4Mmr}G{MkS7eZ6BmooRe!^wBv`;tc!caj#wOb?@_1shdb^Z8X{OsP_#)Z(jHI z_*VRL2m50rF!7=#)W*BhMFKTnFrns{=~v0m!|%BtjP3~1=%By1XpsYJo`jDpV&`_B zOtasP=rr6(8PJ3VZNhzIz&Gu|W}@pGv9 zqqfkp%^#p*_x@LbcYnV~)oaxKW?yvF%LkzHSE;aCX@+{+0*e*J zVUmQ*7G7fnPliXzw;|;deZy{i*>bNs)wU!T|BA1edd8ct0L=*#Ykh6PhL0-Q7FXU3fCtcLv|oChO4!2eXH%V z)m4^5)1T^n+-vJC!`_d_y=Yc=V^{rjYTaH|M=9=$nZ5Eulj!RS8pmy9+}}maa{B+q z#&{b_0=Tmt!bCp%_*b0ZAqCK}mN}RgEva6No5Q#Q9!W>HdQQe*w)NqM8EbyEAT+nU z5MeL>o@%z2_a7&<2Uvaumbt8DlZF5=s=;fb83Z2Vt}LGuO9ksc*Q$Bm;jB zszg$r=LIw&Af7^B^$8>f*gHH5du^6=__V|4cMaXWzs&Z>~k=~fMGW&H-%%P13XLgX4t*wC24hrT*4kjjHJ*12gX1- zzo*{v<;Zq~Ct>#^ciu6;d)q(Ui~g8^Ba3Gs=sgt2$3BrJ%sxedkqtOL_w-jT)DFUp z_uFnl?)VysAo-GO7E1n*VRlcI{m5gBc`W6;i>IeuTlS{2zyFkuw)BeVWLNBD?j(oD zSnh!h9Cy7WeU6b3)54?$YbMxaC9!y z${W@yFv@q)@)*xZz1m3nZaPCAcPDbE@ie?A_;ef}Iub|v@;#-Ge|z9STrXpkdbl3K zqnh~+)Z6~?zBK#%=~BNE$80n40SmtGcJ()V>svA5G6!RQ0_GzgiFz!l*L#<{IF;J; zYd5`}#tGZqX?~1jp1aBI6?f*VZMW!ycu#1VE`9qs8o%p!q&V8d@qYw!B;P&D;NbXK zW76H9d|2~&93A^Gj`p5)uI)JAw&rO(SYq?YS$KeB$jBk$<9keZ;QiCwg?zyqm!?Or zdtAHxyXh-IRp161IhRQ*K*8t_POymKC}3c4R??7-LJz&57B+ed2tJN z-g6FNYUc+bdb~`)JCaYmdajQ2?zVfkM{&o6toW97T;%nI{4e&v83;P=MUaYr|5gOg zZp8aNBQdze#G_~=X=Ww4Pl{{u2Hmnl7ZV6S%nu`qh@P}S{4 zaPHk}mg(=qIAS-_vSLTzlp&@+a81~&T}{3y|M=t?+8xWlAcM05rAH6!PQQZy@be!0 zR{HtJ9_mOw0#C?AdJym7jI<2zPmMGzzHLNB0ejU`u;eU*l5aw=n(5P9H{xBb76iQ% z?<2kIjLXuuProe9_xE=)$jiR^I(7gvD9Xe>yA#LgGL1riGV#X~ce*ltZhO_|U&VVt z8GL0r1MePXV*h`8>80tGGcHN5L%=*z@NXL62w?_{c|0-G+Jgf>4HokIu;Wz$ zFcETb0FaAIe9B*t3otA@Jjnt|Chu_4v(5G=d)0Qf-~Yr&T2E#Gc=R81^b=(RFPITl zhf3ki668rp+5$>H`Dh0racmRyvJI8L)JEBDb)~OreiVPjo8`zSY`4oKEdOMh@tH1E z$OeOIqh!-e%Bi5_kW8|zUPV?#TLf8QW6WR&e~Xr5hWz*ww2WNj2+LP~HrmVpE-_Y+ zazzx7wB&bxGp{yRezG>1uWEjF-^rh9cg0hU*`m(r)fls=`&G?wKhq|)nRc*E2BV(> zQXtVbhLll-Sa4C(Rj_>6BH18!lo+Fqm8=(3-;tO7#ya|q1O`QivQQw3rgr$o|CFb1BmG=qJeJogyfm=qrb&mtIB+SU6y^%E)98_>98g!>rn&o-e5A0O z$43$^v_ILx`@`jvxcC#RXv|JZvO4CFL`y(EgO+u9sN_E(nIlrLpv}A#h(r=kW~bA# z4zwmh%3QDMP#Z|1e+Eq!X@ziJiLpHWAbsYaxU)1@2p4`L9cZN1X;BudL#fc`MA%t< zE=dV(mrRGqQ=j#ToMSsRSMG5tf5Pv?YOFr|`A;>0}-B!oXH`3GA z*3VDT64$hoJVl+!WSz-ZrntyY#K{K^&0%tlF2bSGVY44u=Kt?i;(OPIBhYgB+s{*A zj_=`&_qc^kt8ZSXAO(*jWT5T9uIgl|TRw810!dsfzu*l^vHJ_VA8)V)XuboH@6EgT zEoW=j4&OaF0SoNZ#h}aDZ)1I{*>-fxIM6izxUgO(`K_#Imxs@pxMS$vPd}0-V>j%j zZ#^ds!7fj}6O~7SG9BD=DDhawNWJ)EJ9bMkh`@mHl56ItU3k}Lq+Wf(#cu9gWI*zg zH=nH?gFJ?I@Am2;_mJ5uJc8pRzx8+bXjkk@-*>r=Ip*=m++E0bS@IZR9>+>Qj2JsS zZNL$${62KoCzmOB!6RlP=rbO>Cz;BAWPfqj@5qTG^&ZlvH`z`iwEaz5aZ&JTZOwfxOR&{Ods!~9FChYid&ms}027FBa_KQ6{Qg}bG0W3ktJ zw_CqL|c=g-c&HoNJQB3>G8 z7&mLQ-W5&MIS9UnZr-QOG4tSmtyG})f;XIj`|&K@XZK_NW9o+ELn^d$Bz9$UF1_{d z?@hx-4Au_WA;V5exBkO@s^5iUvpH`~#DhOqyz>IRU;Flt-mfE%xkI*-A7nsriTkan z%31E-J?-jwc<-yN(PzZ>FmuwxNz)qLQnzO6jOl5|j&1eIeyQ()qwv5RCd^2`HRn`u zT)ynqv>ERX{ox7I)BER~rn)m8xG5dv4+5k~*nxN}f^AH{y5^DenH^QoOqG6_i(^!; zUviO3-RQWZ@vrACNtcb8lo+f$_ko+c9;N$C>|y}!wxt*ADA@nrwkiG6I{WZ1?e3O8 z6~VQk!w%A3-pwlI{Y&2i@WA--6Vs-R)x#X^ zoWjq|554fd8jAM;KWqte;=+ZVG_Pvg(q#tEKv3Q(+QfO;I2Fiv^U$`Xm^bcg~}s=0SpIb`#aDTEVXRu zZ`a>ePmRe6qAT<_Nz^|0t^U+~!;kcL_)q%+D!lq7^OyC|B)v_4H!Y}_%^vqp*={GR z;p?nbsq}L&RM>%YgI;7MllqH0i7LOA z^FldZxwu-RE9VL4ukfKHl?^qBbqJ`n%qcnmy3~LQk~~LQ&a-C0__^(s=p#E6(ZZb93G= z`onJ_kFZVROv1$(n^9m$X>iyDz5&E!$tAm0mYg9lU|n7QT*}g_OpJ2ETB<3uc2w=D zQPvA?7TLS$ZSNoOkgrYZD6T}v%?-bI5LmLcQP)z{-d-uJ(RMJu-qFrh^%g~x76r!k z%;_#H(KU@$XhD}15?8ONx2G=fAHM(7u!i)V(;waRzVzsp>Qo}$&(%B&zL;MK+oaN( z8ag8^Q@1nxQsYpeHe6fLxh9K`&yA9iE6f_W_1?(LWu_9O+C~v|<((DtsC^E#bVn)I z^DXjP{8^W8^JyDFQRV#lej9aNV136Z?C1BwHmSwVplR6KN>|z+OciXUuW)6KbV`{a zXHZ>@m$KG58hj~T_wGk}`V4rs)*MGpu;V#v_B9GaM-A=x-npT8ALXIFhqVY^va$Z~ z0R*7OCP z2Wb~A%VlczqXNm8uX4YZ#K)NLWLG7sq0_Ow9dRA}&}%y{{K+%ZLY%D7>n`kG<@T~p z+eudU?Q@;D`t^Me^ymQ|ult^v3q{Q%QyCOxVvsCS1VvfiRV=iRAJYB_0tO)e67Qg7 z;_kxhpSG>-s_@G%O`WIrRmRb}42r&Y?W0|TqGWw0cD;7O6a_^o=R;U@J>Kaz)-F#U zS$JleKYUcW$pd}VK_9$#-J{1_o#bK8OFxbGxH8?aV@nQ-viw=>3Oy-3n)YI6Xs4hk z>HhEZh1sv4*=m9ANxsXMc<%Hq(_a4~H`aHTZorPvw;&L_aQMjd@5S!Y>AvH1GR6{* zUS?vD_~LOB(s?*q_*d3HqPp3B6>0#{L?Wev6j=Kk9AqWfp z_AV|iecFOWUg+X+fkhxP@Lc#&;4%Vf0iEC8y~c&0a(SX^fqui{lHZ+}QT9WLrZI7` zjmh4qWl$BYFZgLAS#&%m35vN|#ViM?GQdG3lB*Mro#-QVsX+tEOK z-Jf2sF#x{YUwH6Eaza^)zYS6I5Bwerrm_bl0h|7g`hb*Ex7*~)h0KDmb>W`e?&ksg z9pkn1w~c#~9Rc^6J(M054x~`OK|Ntk#5sbl zCw~o3Lg#nD8dZs|>@O{J5@!-j97-YB6&cn1^TR;nHO~(*p7VHhKYD$ohy7HQ!atlln7WQv5XW$}16@d^cbUU$0)TrbtqewE!XL@{-pALufFx9CZLbTRuy$-g&++pBrq(7L*Q z>WyZyUe#fMljr{Tdik6BIBk}+kq_wnHQ*-J>N*rcjNcB`{U84F4&_|L6i~;ms=_5i z|M@^E-z_S}#-+8rb`Mm6j&w%Y?q=RNkvsH{aLe< zQcM4Up~X(q;@yL0y4(h#eF|gta>D#>(b7My;^jxP)ATeuB>YqMXLk>{oDLxF=YZDl z@I$nFAbUe@(E9+&ey4wctNOF-G2*No^J`6=9AOHw_?Yh@RA|>1emd57#8&+olS&nd#Z^xPY#H9(76Ijb|0iAzdNI_7b%{-2lI2}t{j^lRUD?#e~8gIx~{*>B$DR+#r zvi04fkPmy?y<1d5!cK=a-e{-wK%DEgHAfqYGlEW?LP2&*gvp@(nGh{Wx0mnI?XJXzqrk%yoBe zN{?LkSi0~>@&3soFxrz(8Ue7<#+J4_vR(bYozJ{|rEcciwUzY@PBep}tfzkt@|~k? zQq$&VBf!zgt~TZ*?(*FF*bemDkb{Lud8(>4&(siHpn{3?o>}c25pX1o3KNa z*B^zgJ=)k!XIAc+X1_P_$D`Z~2vg><1V#B!1=Ag$cu??}TspP8S>LT+yXo)9cNBft zP5$x59aVpz4ejR6XJgJN+u6>>b^+1g>4a-%XS3f!24P2B2S1d&-SVSxtogV9xq1Mw z8|`-s*AI+-w>GGlE;`bc=t1FT8}6=7Ps&{r137(UxG}&hqRd%Ab1!- zQQ~$SKA0XT-Yxo13(ibKQWbdJ2N`d9{F&XHZ=W__X56)VM_P`fek1+BxM}INlcz^f z@Y8tD%^p~lg`3~2vb zaSU=hJue$OCH>^dvxWM^Q|r41P}}iG-8Xh`OZ+R-?YnlQ=?HpmJa8cW?dCN&o_S>Y zG3a7&`_*{Y=_j^qNcZe}I_<>RyKnc?Y2k=b=^vh4n^qj$oqiBImicbh;do-}z1R)= zqpR;NMfdZ44;&v4aP^1@iwMf8jBgSuaZwA4A6j6r_sDT^-8gLVM!O5e&w4B{VNUhs z8`MIM^Or#c*GMi?Q$khRMRYW!NVdrCx>3)?AU!FsMIq|u4@z6Gp>6z;X;0F7z2og& zmuy=ZxTReGLCVFFSXp!Zv?n|Rp_T%Akpy)Y8bKz@{`Q(WarPJeW(!wr6O*@DtNtj; z&uo9-MN}CWhSDBK36yGCQ6QBq_zYe7O@Y~_zuoWjTT9@!SlBYZR{qwQ%6Mp!zo9%9 zsp_WD6s2$^V zn=P<%(bZJ=)fxc3R?aXCQNKf078(4ZAo#otz?yJ-bKE$!h((vjtMT_bKh(!7$3k@< z*!%(u*rr&71x2Yl3ejw8*ZfHTxW71fTE{CXkwRD&;1$_llmj}Jl?*Bf%`%XNCJb)@ z=K0+0&^w?_fL&c&saA3*k`#p~R@g%*cCQHEc(*8nqjnGZ{DAK|?cOb#{hRyC`83`u zcThj4Kn-^(q)}Ch>Cud>>T7m;HQA#%KD`zUowM0DAZalH74CV*(qL4EqJSA|)Yh5Q zSx98IMqgGK!h#ycld(BNeiVe(rIL@LMzupA+)l_<14A+I`)UN0=7}X&a6Cbs#MKncoEo`?@?o8D})-f`6KEyBltT-p(jtYVPm#Z`lzk?{t24mb1ftiicXL; zmHmMuWUA<_Y**7=wXDMFUBYl^`QTM~|A-Qd?d2N6zQ35vb~-9;|I2b1Iy|QEyP3G4 zeg}>ezUyC)?j*DCrj31Fbr109(J%eRP4&ut>3_8c_~6Fp-+kl%Y9I6)|J?Dw4IlXS z3D=l@VDyCRlIMm&{1W_;`KPDT@ZM3TUs}7Yw%d^J)M%UdOT^P*{>o{jw@sOyeq@TB z6V5deLkiAdFt*Qhaw18eFP zjnW@IxjOML(pR5O=`QR_{-;IfsOrjblN1+DnY4*4KKm4x6$1eUhMU@p6Br=KF3_8IFn)O)aZyQn3A2iGyMY=3o zPNX!Z7~fWkvXQvIqw$qmNh1d>D^u3uh)|^>FHS(f6#N-))m^V9)5dWpe=AA^DGXAD zT4rLWY3L0_VNaC9ZZU@dK)PLWydmN5XjtwK_q%qYKv*?iac`mA)E9a)EDc0H_``oe z7jlB!AQ?#E)*+wm&i04@j5Lqep63TDaA6suCOb~*TkJLt+EwLff&4^!(NG(`z*AOC zs98r-rCzq*lZg6gY5LT>~3x{~Uid3ELZZfnlTelaVkD6CN z>vqJBC^e2Ee4N>LCEbTRj{sX9Xyb34V3s&IFVYlBW(01KRv!Q~prcB&1fJhwA;?&TZXHD-F3 z6aES4?B>>*Tr-MyrYSdA_YX1k_D4SkEV(IB{wIq&B+CngL(?k`UXV;r*`KV9-yi{5 z#}vbLkoRBsN00>W5BvT?ch}qDXtR@ruorn1V<1sh*JP67el;~N1bi+2am&Nr&ixs3 z>o5ee*-D0v?f@(`(#t--`aq2xm0@o%WB5(;ePC)LDbgPHPo;10OIZ0gWTz^G;Ns;OgJ+X&2`Y$kUAk9kd%Mc8bRL zihz0CiOS7_a98cVh^L ze|&s6LDi3$q@Vd7c%ah`qVItzeU*I=^gYn`K;Hv>51ddQ=m$nmsP1^~81sSIOVXuy zuP4)=Y*?ADMIfq7n+{ZGDvZN>KpCuLnmp8w31I#SKX#XSrq^Sa-@9fkRQ^UB>-zqU zE0u3|;qj~dy8{8p+jehHA3NpTGz353t{FQyeH!mAefOGW{=>$&4<*}mdMtKP!aJwW zlc0?U_NNun&RJqevGc$T)t8`R2FRtw>+?SLo`oRa{Rm(luXHJN^0-;1zi;do9Z1L0;bYQA7MvxY)5bTgdH_GY{D&==pTsWL zNRx)tr=f9!txUe7O}FgXp?Ar$uX(I)xAe80Pbn};TE5$MtB=8MJbhf>14rcnE*klQ zg%%XJh@yoOgSNP05=F`N6Bcy=aA;MHg%)GeHIrdcL{ILcLUmef_btN1?2N86O=TFgif~215y5DJ8V}+mQgm zmv*yllvAegPgMJOjUwlVyz{=@{17`r0p(AJ7SzPOgJNryAq7%UheG)(dy08*M{u!2->q}LYoBsFf03lZSr zghYiqOe5z&tFd4#35`5w>I@KNG^Em90ii8TXI*@-09v57kXtATX{w@q8gJ6x<^lq@ zmo1?!43^jfLm)5FncFPROw+-MItqd!@NSEMO8O6ck`W$rWCf@;80uLxNe{g?%q-?N|wlZuVH zOxuzPLX9NoHz@ybo`LU!!dR`1~XWn`#J33Dv-H9H-pKF$sG`8K|97_&Fq zTTNf(@0XB^a@=*5NNPfoUoFd50r9sm-!YoM9~!DQSIACxLp6c2?a<==?`#5je3bry zp5SfYZjLFYCbXu?Pni*!?4R(U!)&KX)0K6ylchXT3W9#N?|~D`1N<&FcIGG@&)e=8 z-cHu?U7$mT**$a!@0vXt-cB-4l=^YA#-KfibTst1nPYH#=zhGHc6i#k;rOjJo(OGy zF8ASy;NRNV*|;mszjS7L>{}a71k?I9^gU4XKs_Q+L-h-N4;-5Z-ivoizH*|y^YQPV zd^~*wJ2#u@@q<;M@T?JI^AW2{hFKtv^y?60jC9So$?13JoTlPEyLYCyV+X4LBnQh&DugOGghD}$1Y**5QwQP>3=?|I#DunL|xbyLTO7M!8#KUlviebbNG zosPgdWioMREz_@0TaYdvKPj!*w>Q15IL>(yj$4kjvlj4)+CJ=i;2C;=3kh0u;O-v& zaKeHM7gM-k5wOH#9O?;iUobBNk-;yKT7rQHNV3t63==A1CmRu@jU8OJm`)llmXOq7 zZKs?B(l-3(q7$L0xyWJkUIegm##5#iCdEt!3d#J$G1cr^g(|nH?r*b0_BdVu72&76gTZBnf3m;bf3^OQzz+Tn zIix{ykfGY&#oxhG1=l1dUnCEg+Ja&zp*|-tf4iTpM~KCr`#A*oIF9iieh)vg-U6Fp z0`3F)#5%HS7eq_C^pmze&dPCXOoq>NE#oxwwL-f;8uhN%+HW}7(~b;?Fq6$tiVBBO z^k&-l=e?l&gB-?bbnvt86O;nFgQ6cuubraV@1@K-d~t@C0v zzt*gF7eNuHMOSFBLD#7$PsnaixQ(4?YxQ7M`y>FEY^YIH)eJ-s?KN@3I%Keq{VWKY zdLRglA*jJleH6h!hi*~p*vt@YyiXMhiE$%+G6 zLxVz{K)q&nDYs^E*cpOBPElfO)1cTX8nFIuQ82ih1Pgm}b=Vm&+i3=5x|$&1G+t!T z0GDWpji9){%}*Sgc|<`DXf$xD{;c%7nh@tO`q6^^Y_9u1{O)$!_cq{soagTo4FwJH zt^Mi#33o()hUOePYY58jg+A-gxTESY=;B@G3>4C5_Pq{1puI>9@nMeaH3Cw=8~Xn1 zP&M)W#r+=s7o=^j0PSR6L3q}&-9^d|2;*~-E#SLFmB=pDCO?0^=zEo!FLdY$D8E+r z>U$_e)jmfgx-K{S$K}h0;u2PeU-b|G@l}P}Yn0`E$&pn59s^Q$Pobip?R((6;(?(f zhNM^i;T7qjFRe_EeC6?uI!8_%mR|LTSEh%)url5K$=;6~KJ!(J(&R-G(%@l()B1Zi zrTahgXh$8#g3#*`z*wCIax7AgjfehNee?SCtqAW9WlK$mQUpzK} z$AL@!z_b(JnU90W)Ts|gP8g9ET{R~i!h3cf{_^UqYUmO6%`_GOhU9 znzU!@zI4`WPfODl+oqsQjLfnxud2PH0Oe8=`h}jyZmd9r`6wVT`wo? z_}OFS_t8^ErEP0>q`N-;Q0n;3UdWw!_Q`3{m2=arAHEmw)U}OA(pg-d|Bmz1h;hSm z4c~G6&A+-!KyE;IByUm^N2!WBcCfz9#O$y~ppJ6%0^6Z_EU#{OIQO_yO1I z?_u~s^;45(r8i8N5x+Up6ujT_Q`nt52Je++|NhX*yR~ySY%k%RPa{H&^n;V8r|Cn6 zEC0@^a|L<1erFZDeg)oj`m>YgrtyR9y|C*J9!T?tjna`*Nv|0{O?Zx}tFU{RY2AT+ZO4<4{Y&2i$LIlkH^u6NEs${6h89}62%&|N z3X+bC6oQzq0!l5)TA5}&_(VrplH-th@f)qtcwLRZstX#~7`&3{wg6!?fYMDP$zMU? zSrk7JW+QWKBkKj3k3YwW#xG_A<={_x4YGn|Au^6_&Q-{>#ELc$I!z!KZp;?*6ZzfG ztY^MxoATaf(J?Q6hrv_S8NbV+Eb&t|O)x%#tg(8t$!&r8rpxKoNT;Jbq~H(R&2F}h zNc!8M`?{r5Y5#ZMXjN8p4-RhC!v z4e84J1Jv?c1&u$(Jxk;P9r(YvtD&*GxPMZL;61dkyJSE z3$c=xM!A0qsA}~WttMS{FKR3lZKn{@Tl=ZUlF6{L^oWLZ_GdCtq%2lBB$*Le&WbLT zf{fTn*nXo|HIM;S8;Sv?&tc6^qN#i(oZ8NITfHFj_$%uwpHf>TgIVDUnDD@vi6b3F zRnTk(UKr|7Ms1`VB3P$-+8_Z&BbmqsUeH8~PgGi=sz6w@2B2+Ru=eM?3@+$!*Ok$;1?9NM^vVbl(AmzFq+3tFb zfz~q_xf*3yaQNP z9po6U5=&MWZ)QnMX(wq(ggX8U=v=I`ACR&u zm8MCwP9q%Ukj(x?PK>N`A)>)0Am!?>)S)VyPr?Q=L?0?oCf4zvplLKQGKASsPg3Ox zm8TzA*SNdv^P0H$BR&1(kSxp-A`$8j!c<4d&m{`i=Mf=sl;==7$w#^Vy{3Tr%n>hf zWUoHw%wCQy-Ti{fqa4sDbTC4vBp9^5r(O!JzT;JPvW|9Ekosh&Y^F~8ql`+P*ExdtE-=>iN}S1|E>e+(0*IHJuU)|LC70E`to%7&)O@qj)A=+ zOWB7EreE{E%i#AHrE?JMoHW0B9onRY6Vgk5?Ggk)C#QYe5DZ0d^Rl;{E4hc^n}d4} z<-gIBN2F;8qS_={SAFL9og&$Th73+)r;bXOz4hEQ^PH3GL?{g!Iyk-Tw=P$(bN|!( z(}GK9rK^AS;ubnqEPCOb^!mShX?o!=UzFxvI6a*-#OC^T#PcqmktQJE%78xq1`Rga zQIkfbtAFL9H09KZj6SD}e*BEI_$BisV)CMi>56w=kS5Hs%|K`1p@8$>xFq9V{ei2} zD}MiqblTPP(rCyjQ;Z!3bZ`Cpd(#y7kwMgc>U-cs@<4yb=!w)D|HEyWh2T~kb<3vw zFoI_ME7LaY;=6d+t=QFhBz*+$$^5w)^V7L_$K&0nUnVl{GGyY>tr3j-Al_3My8!vl z()-SQzG%wJD<1iFx)Xu0`*uIA!00s-PEP!bU5TV!gWZ*1dAfegZ7GBKqp^$h^M;O0 zSKwWiO#ixNgZNcA&X-&9?#zex>{8%2+E%J~2m#}}kMbR$C4V!|;42gT_fK2fA9wJY z2~*R1W-Suk+Jk$rYja2XP3#t)jNmBKNCZc}y!ayJ%ggVtUm=+v-1J!b`Co6Hfmv_1xx`WqqQzWRd^7E{bps@UGe9qT3w+0$b8U99{#bNRMbz|96zPyde)zaAdoJWMMwXJp96x@ zQt5AUSh;48vr~NdE$|^F`ave3GVp@DxVcnsNcT?&2*2fWhYVz?IhnrUXZL%kiGIs= z2g1d(SSHG5a$sho5O&+0+v@N7PKySf{RKs4cf$ZvS1Am|ZbY`*>4H1#Hgf1w1{a6I zGw%$sb1Br(FNUuBPvezJTjmF&X3PFRdv6|f-BsR)9SNZc5QsrSAcQO?8ygve4W4jJ zVmpJ=#7$$bX8I$Elh`X$lXkhOSEiM?X_wQaO;?;O$8Fk`p&9Iz*-mOa1ltK77z}0> zh)Ez22qXbQ148``&+pm$oO|E*du3zkLp@u1&$IV4@890%-23f)_q}w=D+_DlDA5kz z^Ca<;udN0?c|H(7bD=4qHZ+blH1^3y9bVy`h%<$@3@|Q5Ep4{kP=|R_CHYI8ysG$l zPPuWEA5`a)qU0Um&9Bo%8K((o1igB`NJx44P2BcVjGFGwhrCKy^EFP6(Kw_``L1a@ zI%a!}2Y(4KapjSdZJs6Y3gDy{0LxdW(>ID8==A~?dHso5sG|}$_=OW~Hh^~Aavc5k zHv|EB1sZ~fb%H~61-5O9ZQ7+ZBODFr(=3$QE_@a%TxPncU`gH!p9GYIX<)T_|Y|Saxu*b z(ah3|PBXV>ra&=H@uo`(f&cMkogHNaf+8=XTzwQOqLH&M*dHQ*m zM5|jvu>{=oV@}#6#l!9Jw?*eX$1Dv#x}TVA&7bNnnuYg@CL87l53uv|m-fy#zbzV? zcT1!y2YAjJ{&xd2?&p_B7guo|r{Wf;9GkeetlMwnqxOKW^#W|s@k(utjH9SRxC8zA z8-yYIk;o=?v4nQ;BrUbFDj-)HW;cX{WJzeiu6n|!!Q z^u@pBiFX0Lw-#yzh;!aUhpl~FOPrnWy@nOdCYR- zx4&R{+mGG6ya*d=x8S7SZEw4M;aA;0_ZyzIyz{5uzg!I8*JHEmgTL}oZHj#wo5An; z=@00ueYu%;l8pI3{_4jqU-b3QTrR)z66JgAo5sNe%2Vz^KhO15!7u#n&r$Ao<7;=X z|LSKfAAZZnmXE&U<2oToKR4lY<)=R4gB*GuV0{4xT*G>-*D~n?pNP}XWyc)X1)4_Pt~c-w*w=dl6=<7 zpNz=r(2I4yd3Xoe>;c}nv~|A$vE+=%^-ANtM9b6Zd>I14K#i30FOCdEYPIicQJO zmTO-0)UZW4zb=2uWy_oY)lJK#PxLzNTaYj3)LfqPjn}F9>i_L^%g1lNWBK;K`z@HW z&ywDAvGL0CqkrLNmb*W8&+=`5^OegDUw7^DI-Fp={3(|#cYf&8I5quIY@}W5dX{Ex2Sm*~e@PVdvD_^Q;Dzx}cI>)&eM z_4MoXwX5Xw)a4I+mRW8au#KJg2@mUwWmj0pl{>t*_pK-(TZC5^BTG>$j z_jla3{MB1;KGVkKS@U9mKZX;sDL*vj9Nw>%;#(C~g`32G@I7x{{^B!lSbi%uB7)bJn+UXz-K^FV?0q>;DEPU_W)jZ*RWqzPq&<`2ham^q%_KjKN`8HAHbsB@j*sH2kl)efw?A;dzLJ-}vHG^pj2F_2h!_1c8ZSpU?WA6vfnmhn|4{Mz2{z{%CWcI^w6mp<-^y(!AC`Ta|`+_e1U zC;e3q9dNJg2z(w8&;kjbU0P7!B8pb3vJ5gPmlOIC$Ia2H$(wpCw4kwBU65@VuugIF z=!dJ_*>DBrW`#c`&Ib&0I;1Q4WZ!-mhrZihfp7t@7LioP#S280H)L7Vp3;TVS+4$^FzU#`Es$>dCSYx@j?XyAuKW=q$))N)RszzF#w&p;;m$k!DQ;!5CW;4E;+K?**Y0@6iwo7jvCBE;YvH|m z&cI#^(ml`NO!F0Up%#hho~fMne2{#b_a^_$vn_r>&4bBcWf&SDG66YXnGde35C?cW zZ>YK)j9Uby7iS5d`#-SrIU&4_b3Z?7K66u4a(<2Otf$`=#W;aFZ^q-eQ4eJkcVoudv1NxwD4n>M`Z6AAbHZN-i4X1wRo zUG-P?${aOl2^hpHO0is}-_n<6*HnQLN8xCptO52+fFIrOTW$UeZw zyo?iFZHwKnqC30<(3bOUo|!oIHv4Tei6Svgi=v@dHFr?)=J6bQWwiAx1zbo`R{F)s zIP%Vp22X`@Q}p<6i^iF8>DS9m0Bg+=YaGj5;VWX-A@lPtoYQGWre_rAVm)uxVca+I zdO|pOPhY6;S7+l?#gE30`P zM}Jnuo#oe$ic@zQ!%bn(7-eAR_aP%2=p5i_zqGL)94kFt{A>8y|3nztWq= zF_8u>{eyhXaGvzzS$r73V(er`;9)@Eg2!C2e93RSVY%q?i}ZD-U-R8xj#GEfzy{y_ z%f*j>?DECf#QEXxdR1q_O&xAZa-;Kk*d+SlKlbC=XyXRag^yj|wLkU!zqq`qzB=~; zyr}RuM!)8}Zq%v1+uw8N@*Hd^eaUaVVflC8^J<;^`^|spYnCTI;|bal3%!1)FP^FU98KkNojhEmuDKiP&(%DkyyOD_U><*;~|?`*AXn8>3g^ z#2x!~+f5%|F1zwF)!9zpuKBX3EkB73Jbu+L`Nw4Z{m0+-eXqbqAo7Rb=i)0^f9kLQ z;6E0pJxF63u_~Zs4>-T;7K4D6H|LGt8_xc*z z-|_vwK{Db^th4#|j2B_|CWG z<$u^4h1`I=6#Qp><4S-6qQF?&2V zHn%0bkG<2nAAc`46@lp$Pk-FO#^^6I7RpQizzYvHMxXl|uhU6SZj3$-IkP+qr%JDV z$<@nyv6=cFd|mEm?|Wdm+N{-b7Px-JoUQW7~RgD2XLRTT#UKBmmPug34wh^o=>Ck1)j^z_)5~}|H{uiq+a~M z?eAascP)N1lI52AUC`OdSH0~ubMw^tUDyQs9&C#Jxm(_~TzBy$cve5SyzjpIuv}CN zMyFa2;aT#c_V?a;lm3R~ufE0;lBeOv-|76*A9(xnXFl-G<=GcsjGkS9Q?U0wxF>c2V)^LC+G4>^Lc@r3)wB_f(#eRF^_#p&%7>!j9c+qHFD6um=_+r^S;~<~ zEkIgQ1~DwE{g-u5u* z!M=Z^RC7TvSS{mLVUyQD7=j~slSkbx-mw?OrPsa6e~9NB?Fsz`Z+l+j9-KqDIm*5i zt^$UxRngSC5yUZO%fVX}sK5`mJ!NoHAdbG~+#tl04J5PvS?PBLahfqYjrq zouo!IM*xq*!2Kk=SI=wAIj>XG-@1r{3&YRhdcKy)zJgFJKGj7o3$hf$s zO*Ar+cunk_^jmKPmFYxc7V3=-j@O7|b+gd49nlqgby#dMe6K0u}B1 zzv5}Wp^xpiCVu93%zIo;z$ZBAF@wO}ox;Zxa?A(j#van0N_W5d9o*rFIShQ}1Q^wA zQ~O)_-MTNo97pM;gGDorqVmrBJNUqv(`#CADeyYawc_VGj_)Y`wy2z(fKRLs@q~oJ z5_l7R?)L^C#eqp279J-NqrwoLQ4WLt=XLdjd^>956wPnVX5Jmn_qG%sneUl@t9dsi zJ;Ef<4rr~$2maCdU)6ZRlisJwW9~51;7MGFl{fn!%ll|jl#!>Jbsuy1*l(12eHyrRPTGGUka_~O0qw=FKZk%-=Vs`-+f=|{_H%$>H(jSwk$1j z;4Ru{<;LrKUW4C3{X4&`jo+92o-e_P$niCv8 zd3yB?|J)mrv&oIW>jik+Ke)W}RqvbGT_@)J?b4L{KXX4$8~b%7JW`$(X1V`dy=73F zZPzv2Qd)`^cWG&h6{olqC{Umf+#wVx?!_ULV#SMlg1fr}clY8N2(AGV{NsA>`+C0T zeP_>{$4t)2zwGRNtiASHv}4ubT_)_O)~NyJ4bZ*vTai}X*?W$r^UA@)ECsr38SM=_WY`HWZw+4bT zq6rk+{aps760{UaHrzXnC$xdt-fO_`&V^hS5dWxc{?S}`*%q8`q6Nry`?w4H!U!v zEm2?hngCZF?MlYs*Po1g9$7Xp$zDLlmD@snyf`-88Z| zn%QABMBaw0%exn>l0Q&-xh5~?d6m&*{x)RIlhw<2C#i~B(8&M3kJzjB8( z2L?mKTL!xodU1jE=F07?ITo-#y}biAQr@>zO9sOOFg}q%hC5`BWDm-UB{-SvTni9Z z6gAi8YLy3|MLjG2@Kq;-(7t>2Ji0qEK*P?c7P4d7n&-pLLY*YVR?rKs>FZLe$smUU z+5uOpd>37RU8FJ0iXwmIzlblk=`AIu|MAH-RQ2PhfCT7MKf3&Hx+}1}ku5AT_9M?F z>ii3}r+P8qa?&*G);j1R*7h*WQ_Q{f4cbCD5o)sZ85WJLzmmJ-8TIh&TK1;pG>O}% zJEaE-cQ(kWreTGa{=05>Z!vnk>r6qZx$1Juue=(}-9OdejRl{@#%4)Vjir>|s^8p- zi2W&t7jHLqGO;gsj{eM89#?<=rr4CRLtSV%P{mYm(OWvQqcakz9{!TcP zY5Id7(wX*Dm~)+^qDS~aemVRfH!Z`rOEj~EZ*>!GTwDf&Jw_QnM<|!6DaA(Ivd}uO zHzX;s6iB{$f<<>B>q;sMDIpG7{{bJV!g{&t`gvqV=55@&&}fU>9_xa5_}>Lc9~Y5* zY2s<}GhL|(?Fc~)>cQGhv+!)$6qcpmd$Cz&Jcfru3hO*22Y-orMZbK`^$Ytj1YN?T zg)rt;sD$i=#7+FLF9v8<C$;F+^k)*Kl16diPKQ>_*?w2_$>mdQOtTtnJ%i7;R?jg%kXD zVQzarGC|b@p&U2<9FueawaHa8)U_^X>nIGg?VL#F+ec;{8*|SNS~4-OH!t%&&jWl( zY%9M0fr7La*bBb*8pOT78(uwagQ~hNRe1GA1h=_(z|k}M%N4gU2b;wX%02HkZSVJ% z0HEnO;MU+&;O+IwDr(gyUMXtQxAh1`GuQ9^o-f9C8~D(>6^mLh0Nt@a(kh13fG2M& z%lcj*T!0=CN(C{e!(>tA(Jh#%<;>AYjThr?#AqNVm$_z<`W zN(78g6DuViUkhSra}^A1!?<2+x#e$f5<&e0J`-U7aK$!0w$7SGC%^95>A%vaFZQdP z|DMtJM4TR@?Yab5%*2mK3eihPoi5N^~ ztWXmO)}Ym8;cmoaRJXsa+h&_xXN^KeOwcz?$x{09*oKSUhq5LBmR*jHwS9AhtNT_7 z=KdQ3bgxbr2l`U=*)1$y%h}U6ZvN(P)a8_@TRS~`C+#t`i%gz2>ChlycW6{s3lq$J z+usqK&LRx=9@%W z4AA>#KLn}$dmVvd{}9Hy&LCSwLs<%s)JLzo#9cm1Y*(wC)y&hltCY)-xFBwJ=*QrG zXFUYaT`J_kd};98ww`TcmWS3{(iyw|uytfQaj28XbQvbxG21_aju*g20z^-AntkuA z&Rr@Np=T+NFR4U!(S|ZCS+%$ue%_Vg={q339Y%AUEG7=4(0Lw~z4D6V)}=uti4O;?Qb-VCP7QKP545`v(l zw;8_wxS{0*{%**qd8>ZRV@LuNr~pId0IX(=Bn?WfqU>8c`gVAdZ|Q75x7*wEeKuM+ za2ntJai65M6USfwx|FJ;oUxmpbPFHvMu7QsE0@C_It&L57dGF+s|FO$`cl67- z08TZc<-wi*@fV22Ic+`UfpJxUU6=DMO+iX*Y?6;jV(%ro-3ewoLS1$*rb5(Fe!@*G zTj&9WIpSXh$g{mA!HF=x(Oc(Pr!d7P+u(^}=>xk3D!4IWrhTGNM7nNBjh)*5al{#Mw}a$&%+F8##k;wJuZQSP`JV>(oz{e< zsUK|P2Qrz{0>T-b&%9+dXU5)~Y;MN`oH{um)@mUj7jwy3{LypGhA~9|-oAcQI6o0_RQg=q7bb=oe4nP(n73_ly>iKbvLWon4-rzN-{}s9wl`6@CSywcw+mRBg#Zi5ON4 zr~dX0wvAUG>uYvk{HuWxtrugFD((qfHjMe|WuEJ5oH85}z3YI7h)3JD6;v~raYgSW zzL_!JOWcL7b>Z@q(0N54w3UnU(eX7qv~flruIRU(H8JiW&C9!I5miffr(J@=i|Lwe zl&N)E>(fV%>s>dEH+0%!f#hWRD0Q1&AXH?ZwJc$m7IeOiRi;38Jy&!8IjBGE*Ml)X^U*?zg zHKt4WN7nCUYKVn-`ab0arX!znsJf4?#e`MB+&(s`^ky~gRTKqk z^9>GJKn*z$>1XKNX5D#B0r#fLmHI1uL5hwiK63ghmm|^%TX|`{>BO5)%Ql`rSj}1kMg2Def!>O=ym>)VyUxnwGA1^CQ{77m5O1SnrQj70 z+dSV2MX)6paa9Ep>jn^ZxVPKs5qnJ?k75j#D17Zw46gmn{ycQ=>We6naQ2M2dHe+C zo>#?zKe5^%7hs4gyR9x*b($NfIRJQyQIF1gg1$0-zDSd6c4+$vIl4TTO=DZZ^^Y;a zN~Sen?mb75BNASu^Lz2e3%n846qsJycN~u;7`%vY%IZ6aJY~VS>dH+SC&oZm=U@07 zJbi68%?KKJhYx#?n8ZlNGTXI~troW$XtmJOpnv)CgGEdcy)uMcv9&$Oxh5OOIW>;i zzcDp$q%3sA;n2JKDdw0O1`PbYL{>rQ?4zZ2N9bgEL~&JI25g`~^oMVbLfu_eIWrYf z`DwmG@~e=hY)LbBqZV?XhXqBdLP;8P-l*+LqU_4lzVK5S->j#h-nAaXP%mmK6;{=N zH*Y@bhk9y%|6JlV8J}|LZw=;;ty)$RiWx8t#n00@gWIy`$aBJoW@lOoiQ?i`&wAh| zstJ1;+rVk#dG}E?51Pnuz?%tGx41$>**remZwvJ#Gb%whloOu5%cr>k2`A{ThwaDv z*LUUA8g?gp?+@4bor}#eTbjuMvp3xq*U?B=j)u%gBeD&{qDvq|3kKkUcmDR=uyNQ$ zX0nR%HVI@wG)*IMre&*yp4=RM3Hg%TEnT=02wh4Xk1?G1JI^YN7R8BnyP#yZdE&-- zeD^r*xd3{Va$m1e`8dTzP@PagZfW|qThHbbTK?uMJj}b@W7GngPk5K@AFNj^maZ}Z z9Bn7QtprqFfX>-1L;vzgpMx_HYTcg05Lsn?qEQbqbeE5Bqw^fvr%q*j#^LoiY<$!~ z#>H}+=xG%{zS;B{ik}se0QTIx-W*tc>cgir?}L@9&JS921}JQQt-^1;?;46N^C<(d zo=MqXGqoOF25lwGf=s`aObsR)n}kuB9bW`fW z<-W~#OI+4RdW<`W)t>>k_df68ek%q_11!J&>OI3f#JKV>u%#xjJUCSWJpVYR|Jw`{ z7JC*(=X-iKxr%&uZMD4kIJ9dd$>%FXzOiXQ)Qjkd{T8cmz!M`#&9g`4+XLfK+S-~fm|;ZhV;)=8CtjtAj}G(EZXfX1o}t}hZ85I#+lk{mh&T=j z`i`X|KhU*%H^8L<2CW;&)+atq)*dW;k_f#5C;)Xa21Ll(P4AlA2kkY&2P%KC_Z{~- zErm!ZmC{ZvdiM8O{;Okzj zYi*Y^8=$&oFxmci@Po}~;9m{kxKXwC%o`I`Rht?Thz~PVYHQS;KT`ITueDpkb0XQ% zoP1p|pJ?T4{<+r&=JL5W=L|CI>3lweE4pUHZ>+?_J6cP38&4ujB}*vJpUIs5s$W86 zaZ~-Q>@o1shACphs0Z%}^{{>?FIuND(p+}~y?I9?$Xrez0_v-B?4M`eQG~{Cg)P_w zxftR4ndV%~4iYA{T~s%T%QMmT5j-mBP)<2<3%+B_7m%dUM|-*Wn!+ZdUw*pPqsNz$ zUp3TW0K)4(ODYa4*?z7l@V<(prIU_bo!d~AAoz01TM;s2&lIo#Ew3?;hG=woV=_e@ zPuAApEZ~v+odH^@d*yMzm5)0c9Q=&U+GheOFk`XNX5`yQ_O!!;ZeFG#NI*NGWVam9 zdH8n=ix4@yQ_|QsSR|@H$`Zc{xyWg~Gp5#fud6)w?KJ9DVFy^-=0WsM7w6L!|Z-Ao){ZdA@WCU+S=t3UI z9heDW^By4|4f;BQxSS*eN3w467<_{3@{&G)B+$S0eYwg<5?K(Ss-f9 zhXB-y?`K9lL6_0|KJi2DySn#nc$c=~u84mq9r|`&`-j-R zu-JU`wxi}Y$+m~9r`@zSn%VPCl@IX7~R1D38l-Yai|tgYrveIk3o9OAk?g zqJc*;DEK_K{))Gl*1Rc)ET_qFL_qp7&(~w>tQFWm(pDIXTQ2@M{K^G1m|l&i_6_ua zXp#)F?|<2YE}ae)yqg>tge+)LpRD-T>-ifg`f!^#ZvIl8r^Y54I%4t-v+Elor0jK% zTJ{ZXKee7CK_PsrcK1;$KYeD4SIbwee6HM93d;9=E*`FRjV{;DW_)*gx31cD=NBq& zhG~#An?YA%xWDRwI#v-}|n1NCS8< zm1vLhU1{~+NNApZ*uD?iuZg}Mg`8*uqj0CTeWxzYx?(fS?_})_%G!ImZcs^Kbk(;@ z_|T^u3=q9KF|+m|-t+b4pAjgsCv*u>E&GvMrQ+9iHgABXVkfN5z#IHK;jClO1>4Y# z@2RGd$K6@2?|F{^iB}?@g(VUUxuJ&-1}B)W6Eg!&LD$)C9Jjc$jpypZF;eI39T;fZ zXN%eE$3JA(qGi^8u|m8rnAO~L%9`N2L+NhUH-mJ$rbv`9+P@L?77gJ2MoiRgA0<0o zsgG*9@-O!$T1hir{|ve9O|?*F&v0knN{aVbVb7#BOkun!?PwxZWfUA=D!(yb`O`O7*6KhwXI!%jmCiR7V$Khl33lCGU$^b*jB&ou$xG;`Cm7ul4YC;i zfV<~9RAEq+-acWx-6y6=XM*of-S^`^GJfWLqnTuP&F#HboraKN_G{GQrM6JOi20ze z52QA>tmF*OjkcUejLw)vsOQ*24=W#zOS9!X*dVw%b>szGL1>J{bFFGIf;7jX&VOE- zWYP3677>ZgIy8IK^%QgO zW3QZ!nn0bXH;Cw}J>2_t#)}u$KlR(}&>+xOWWG&_DDRvKs*6yreX^zPv)q^!e&pbL z3F;=>zqzMtL&zEddSZu=4@*G96%S!j>Z^N477P!WIs$>Q;i%ZY%8_s4inA5?7UrUE zoMSz5Ua_NZ3$2)`m4aynnUuyJRfU}0%!Lhp!Vc&`cTmxp<=cB23a&$#SvoS%LtMf$ zM;(cOrvU}|6rPIT-N9ZuWUXdqE%>cqLFfrR9d1e~qz+#pBfUD#V5^8ro|YE}){(di zhF2o=CpqT8O7DGSi?Ig=$oi~y>iuXgN@&^ZsB<>j>E3Qodd_>JclPdLrkUzAIYmU! zeJov;Z^r>);Q8Db_G3jKWq6&Xz%D|G=aYmr^3aM9xYsZ07=Y4yAP^jVERF!yurN@ zsRj`p7lk9EHwaAbM|bt-eC;BAFIz9P&sy$_R+IQ|v3ACmD#84H-lO-;cwTp25p;EF zgK{VyF#sD~Gll>iN7E*kc?zpqL0RiLKJ*o1cALf3_UZ*}=nhgwCuX~AI8T78eeICo zvJm?wZMXObfP3ZY-_-o)lzr819L-035yfs7>bahi;vO8z$KP%wRLVNtHgVf%BRBZd zv-BnHY?gd-4AJc;>C1PrqHPjn{-?}NZ94E-$BAy>P|quPnAKApTjd)S&CuiUTHDMz zU*~?uo;9;ky(@b%_uU>EME*%SCT{I`@aPV!$8_O^q`H0gIQlg!bimjzd|y0=Sxw_Z zsY0gazKW**Mg!f1b{ttnH$*s+XnG9%tia4gszvLzPzFqzEBR$3t}Uz3jEVJGE2*f|EGC$> z!7}>=QAiV-WJjh6z$~W7(jztn!w8B z6HJ@eFZD8Od?N@%2R><)6}vEe-<63o@MVt?Un^u)etc*jmKUZ(G&fhdB)Z>>*E zXredTlgLig(e#))BSZfS#>#xsc%a2hUciYN5GhX74llUDQ_1l4-+&ggBY}YZUqxC_NcsU11n# z68^27)TsYiIPrrPMm8UqZu8tiVurGKl{8+)ho19zQR*XJ_|Aq$u;H$??EqqU@xIQ2 zHY(>sFe)x+gTCwNr|^czsZFXOFl70+9!PP#SkRd7ry@z)g(??bgxY!x#vtD+2+}~v zIW*|@spN?_%dwg4+Pch|CCavYRUPPc{q*)|TfdGfw%Vpy?&Hv?u6Pr(yvY()pD|+f z4H9y)2wr-U#{aZF)!KYymsj*!WFNDYaUIF-v4EfvxVW-rn?DeRZ|5jho7{RG2~}bG z4DT}Z3-j4AUij8B52kV1*QI8e(jnyTCH#CIyxa)Eqz^S$?yEJ-qK(4l@pR;CnDdZ_ z8~X)*>9LT%WhV{{753p`OTUj#UD5v&w2yoIxjVmfH;v3&l-jo%tbThYe-^1b?TyD~ z)BT9~PW)YH#$}gu(uZ0HuIGm~4ZqG5PXq)5=5(QcVg+J;hSOn!Yl(M*_;2X5L;EPH zsN_kn^xr0*kw)DThw4OSXzI0Szi1`({Pe|aD%8%*MUz*@kI=j)y&^-{hzMVH3HR@2 z=J7#Sdq$A{gBfjJC3~gw6Ofq)WHf&&|1uK z;784ILaBv!@lV+U5`yT@ANS|!q ze8X~4ReY*YI*RonUXO)%S9CGEgNxaJ=J6%TZ>1qz(veB!n>@g6wp;3gvY81s zJ4@kfEAZ5>um;xOS@?NdvS&&_;>L6Nw)A*4`!};bKBld8)NSvIJPdeYl|mG{jGu=- zX5o9s^1;-=yA$!o7TPiV#65XWAHJ+PX>3^JRimjH3A<*ge}8(XaofvuXPHvsi3Z#; z9Ru!sDQEhv@RUo2L^t<~N|VYUJ$&b5J+US0QS_5)plf_;_HL8U239jR>}Rv!-~i9a zM2Xa$eB#U?5dk?ez&M%689appK1(;iu@y*9m#XlLlz`ht%&VA*2$w8rf&yut2Zwg) z%_K>!t9}{E&T06A$8VLXg9ZrytCh>q&HnNx9til@^s?E-VAl34BlC}I1+#iCH7m1QcLC>v7K1sbeK*uC zhLUhWZ2v;{mE?&wwv=rjr(fpXS`PPh%Q&I<_K>c$3c>4#}3-}$=a=-lq!9?_S z{)}$ai?wqB&oaMnfcR}IkpXN5byQ7!tRDuM%+k;!K>g@%{10^;IR-MiTHCkR6XW&~ zJ?Kj9$>yMJBSoz~1VX&dU2Kjg%7xKx+uQMdk;R%+cWp-!$S1;R=AVsL$we8In-?P6 z|BYmVHDd`|fj@AfrA*zOIU9ZXC~JaocwUhb4mGy6^?UvLla1?~?h&ClZ|*;0+Cj7n zRUASC6itAYo~RNZ9ow&_PxCix%qUVt?aIqSvGFX*yGn?YkVfHw3mY??ilh!D!_st@ zxdjQ3NSLVoj_EVO^=TS+A^lb^(4^W*Z)r7-lCH`U7x%(pQ1<+ zZR|j~Oh-KbQ{`T7MO3FS>bVvwZx_qlZ6n)`r=0&3K}?x*s}Puk&G?r@)-!7*OD z1cdJ5Q%C(L1Rj@S>IxNKoYMZ(L5Zwx-cozjli(+6h!GdUmQ2H!38-HWiO|3vxkRYP zZ==59e4cTS*uiuwyBZ8gUR?LCEeDt5__q=Gg>#4hLsH^Vb`cf`wj#NYF(ir`yRcH<+ypEC+*_uGi)fySq%{?E48>ZJZ(@u8M#whzO_?W;a66( z#yu8fbMeR;wC(pUEeX$Aa(4nV+X6z)CL4^Zc?abzTFkpFev+!tIrqYoO21v%X`6aX zK)y9T(m}QDFBXXMn|G+sYlqikshLK2QfAP)@8I5W^;`sR_WV0OG+tw4`rKDdlw);* zEm=rk51<-eWJEsnZDw~m;yD0@*;83rPcfFl4?dm_=Jsh6#_CY$bfA8xgdsIF>JjO& zapUZy6=}=|(8Cx?H%n=lINq+L6{(opq`KP_F?5Ujc)m_1aUrz*b7r|Uwj&pBk)A?p zJ#^(~3_qAo)jtq15;B2(>hKO@bHCB!8{`^egVK$X6 z=!@5H8b2Dp>NlJakXZ3ZSv|vv1N}d>jGF0~>)%>-{}A|-l(Y)R-17v}jxFXD>0sk; z)1saib97*V(!h6RDir}uxq*o@zy5aU29SNVU{qtrb9^?W!2dhgwbX&`=CB>cI7mnh zYOu-$@jThn;q?oJ#g|d8k-SUPNG>*(;rOoN_sBdMR7xNqt%Mr6P#-&HWo$;YL0PvN zQYI_@XmRUEsvQ!CZ@-$A<6JwK!srn9oOy91dpep$wvE;s5@yoH|BGmKP5;L`fJs)# zdxJ&%Ua1Z}MD5Anjm!-%g34{d6OElnIzi*Aw*=z_+H2N9$&!OJ%W~g70z<}_vHfC` zi_Q;|Ozv;QD&XWCHKlV?QIwA~jAG-|E0((6h`|F3`8vXJnP3~c1wR%eNK%?#M(Ps@ zU9&@TW0`N?2WSQ3?@mPN@CGU?JVp!0aT#K^xnGW@X;$c z2>)^Xs%Q>tVY=Qwl}s(^E#Jh%VNqIe(i(#e0^y#9S|9#s29G;Ud4Sb8^Q1)SEEM>c zh_?^|yza)%HNJXrXB%FQ)Q6s}s4LD*1O`!z@^-(w#x+;L+s^0Drz@=~sX{4IY=b^m z(osf7&GH=38n(k?=cVH|&%>gFafwRF(APB@qu-X2tcB{xR7&E*Ld>6sePVo@H0iZ7X^E8*-4cbe7~&kB5$g(|-&THhB2W|I{o1#&>}Ae?MNsCVyJiWc0dsh*}><;`G{o$Le+FXJTzg z*jqbGLB9*`m$||T&%|9b$p5yJh?FXP>6K2WDbz*$rfH zZ*SmrSwu5pa~KKO%k9v3sEx(h{{1{9p*mMi>afFs{X?;14^3P>T0Hv2Kimc_D-A|V zY6WFGN@RhmJ^NukYX_0;8ily#$QWD?W^kLv6b9aGr$5CzbtPqm&e(6fNEVtV;qCcp zp*SURQ+T@LxUbaljV)ZLM3Phag!EZN@BBCU^6>YPmGXhFD0cD0t!po}j=09GS_x(6 zu#c+*BRK7QKHSz0@b-)~^vL!t!c7?AS>T(Je8=^43Ri(+lhBu)G`81V1hu_zp*}VP zq&V*W)R?|aP%+CEdJckpD<ItEa3C0a#f}%pT{w z)#ngZniqP#;2BX56z1hfwo{56bt1Q{e$G1TZKYs0 zp*owZ+&2C6j5j1Uemgrtl2mZvdzxwJ>cHJwN6# zz(XBVlNh?Z#m!>YzJ>Dq4!_|?KPSnVh6%H+`H+l}2iDaJDgL8O3_PKvST8vvsNtM1 zgg7EQ%j{@d(mb{#X_Y!!XLesqnu(%s{7LbMN^u;Rs`%z*2}pDyJi9tiab_cJES6sR`Zp zrNsoI_0K*^^mx#I0~g?B`@fRPNOlb>@lRUW2N?rQ|4pljz)uNM&_5H!?D{`GF=uH+ z7+O?Xux)$36j&;mkW{8}s>D+8(f5J>dSyWxTE+RMsE4>;sygGXuuw1ELG`YmW>0aT zAiLA=`61c_6UWckF}f9gC7LQiEhU{A_bn`p*4=+-sE6=h7CzfuxIPS4eP><7puoT$ z@jb%n`+GB2rH}g^HDphEc))D-(5}_-*pt6+j89xExY#XzQFs@^~CI>Lr=8bGfaAnEnT`)gUKWd(!YU{5vgQEmS){l zkSAJ~j#E!$qJO9rEMD2+va%D=A{-(X_XI0F#XbSXLWZ>#C!)ym2pYin7zh%_C{a2g zm(9k}W}NzlZcj0l9+Vkbii)Mz-dYssOOfC~yAd<-0X%^y$7xN{dcjm zt3Ovo9!FP4BT8$(6;;-?YqlNdf#a!}@%Mbeo7jU^NLG?bMS{<<%Q6RW=1bL2d!dXe z^htV4Y$ZflBFsfZ*|N{lhQC@ZGraZQkdtbN22ptY*rmcb`$Kt?dZavw#^m0*&@e6f z&RBVxva%L?3^w4SWF--(q|-a!h9ER`0=egc=&mE=;F8B zu3$M)y81DdCYI(SA&-j{iE@MU4rE1+^S9Lt>ioCI{CkaI0%xzYBXSSc@Oj;67(AT> zejek`Mm@qPEq0L}<_1mC`J;(GvT@KhmX9&fGwqCbJG0S~dN4U_S+Ju#zNY)&A7^V= z=2=d@zhtsQZa2co#9$y1_*~?qis#?H?my=!Hagx)c{-WSq1=DyDgQjZI~dvn$w?vi z1BIV#dH;+A65Xqm^&pF^pZ2+M{2B*0-9nm5ViTfJ#<5Oyj78t)hv@Fc!Dy^QkPB~H zK8-jkR&CtEwv#W1<~C19K;{4-Y>SoE-Cw`pXY$8K#4wg1TV*7$Gyir;|J{neS-$|Hv zm!~CV*Qpc#9*Mzr^UN|R;GO>@mkus8*_)y-P#DJMdYjvmln?(Z5c>E_NHkm*gBM?j z{#6Y;O3KIE@F;Rmt1cpr?`+V9*CC|Szhb^Pe1sC45>)rII)!LL+`ja%Z!fCx)MevV znzZ(g_?WfzLCkQvZkkX(Mm-Bu1)tj&Zm3S%Eg9x1sx%aa>!j`-g_Cz2W$g3q7H^s* zX1cU)xu4C;$g-$P6^RJ#LahyFRpjkLxw-bPJi7-75-ymTSY5&uzvPMcaLlVbs8)o; zFp-__bE|n8QE9!JVPqkQF#c84q(qCg&yACAvvoy3CV;h^MAkgKL(=ngTkft%4wF07 z>9#8`n;tM&x<^rISj8h{ls4*`Ro1riL$|-HiY$B2 zUL_!W;|Tr6Su=c{WkNf;(N&9ZVedgA*4%=*-(OP4g*H-Sn1??Px=z?bU{LN2D@kIw zZ)dElGqR4!WG>IqtkJzCrOAS_69>!COked#=(1|XCOazXi^&!iHd{)u?VH=t>SwAw zX)1V?cej9Ews*2R<|+=`(h|#?%PpSe+vOSaOi$g&6?&*EfME1|iyYt~_W2PRneXWx ztK>3B?PgouYuTRzB##8WQ$Q6J1x2`UcI$B+1jgK&&uA}3icr)?+0_+5{lC`5Jhs4( zccuS75Ey=U-8)x2q+aj;Mfsx$(a+_az0sBwm@g>af@zJDcjnmF_Ru!3H&N?o;R4^t z6l234p?$tt+UwXgob2a>y-OfTFg7O`i^@Z+g9A&Ac$b+TA#BFQwaqQFo#?X2Aw}s2r4w0z3Wo3lN<;8MPDCV8#$U?FB zH{6w={wBRd)vRwlKV}j*m;dDbrF0emf-KL-)UK0VXPeaz*h|43i_M!94U!x zMMv>@>yQ*08q*Tf+=kq5=RSxKpuKSUBntGI?3~D~>h!T&v{GMl1D`E<-OHtb|VaYzMuY?;N=7WxLFqSwC{q`d|nB z&EIDm;Rfmd^(O%qKr=!yHkLTkFkfAs>hQvY(99_G`ZMT=r0LP{s7)ymsbm43HNUo1 zfsdSK1FtE}yxMR>tf^Jw&$ClRGn0^cCF*7n^Vs++Pgm4p7He@tp9>Ge=MTcZ1>;g{ zHeo58Rmmh71*ZwiuTTMGi`g|btZ_Q|&QE1`e$n&`F`%aaLh%+|iAbRf92A29MCxfI z$zD!hw|4(83_Abi*24~hUJ)gcFDkZfRte%*8x_-xtj)C^K%(ghUFrD2CoCtE`11S1 zAf2&8L6NVNs!aUK7Hw*>FT}{nc9578YuT+6URP#H8iPJ>v3Qj}^-4H!;CJa8#>G-+ zzOXv{{DjRXc;?aGWjI3);Z73dYQKP2h}$s$(>WsaJu4HL5Z#)7X4IovZ@)PZQ41<7 z!C?$mD{$4Kp+kKN8D%Y-NIxl-2ziM{)b^UamaI1n~WT zdbyuJ#kL|MsLS5DOLzWg7HjVbRjH`yy`Z>d6fHIVByV^d=ah2V_Yp5ad7xu|H3PPR zb$7neNH)=uEGt#7<-`^+&W`;#L`EFbf|g9NKO`vUOK)35&#-pY4|2u+Vw22EWzRhv zGLNK1%xS-()p!#3mdL&ewdm_bkEr`dR`t3_f<4BzP;#NZjg91w%?Y2E*{Yya;qGC@$rML+@-#dJ!O7|*jZh6 zZ=(=g_~nvEYnRT3Qs~ z3vNMsg25CW<@NsGf7V|ga=eT~+xABssH;D>Ia>*2W|ICHc@emiZ7JhNMj;k;U*<>Xz2!3E1g6^ME zV}^?_NBSpZ_Es7=OP5b!wM3#84hdQe6cHE@*Q;7&6YgBu?wST z;6JxtcvMih(mXI!$6Cp79@BDH(`=Ypx?{J0jr-FPzEOGu#G+}zO_Mv2?0N3qo{rD} zl{7Zh2`x>SM9FGQvQg?Bc?eE2LA}6<`iL4htK+D9z~9@;yGr@pX_*V%0j5W~vb98V zK0}6HG;b$yF89T!k7F1#Fb&w#c2m_$pI<7p4$-28)G0DDSi&hXBct!1m&rIO^BxWR zM|wkI!Pc##V~KND#DgMAI1+74X|eP+(Ph_Tm+Eh4pATnQs+K~cszOYqc-AFhOh5l5 zh*}S`Q&*Wo*?bE;wy!eiU;cCz4UQ6{;|A{M{DVceg4E6>!vX2UK7VTT!*aMLTviT< z@426R{7^&kIX2xr4a+(Ca3g=yN)pI!%(%y;rEC*a=Bvm1beleZc7Ig4Q&+vbQc;LC ziVd5rhL~C2jhX=b5x`ZQg(p!+1*v^%s#2m8^}T7Og&yE~`8m~xt#2`c z4{^5kzLJr10|XQObotD7z!qN>0bfgjUkYZDgl z`sA)`e}XqRLb1j^pSX+F5M_rXKf-d+=v+O`3T}MxivLd`76AdJ<=T zr_oXM?Z{lk!vVHW3?XD=0NezN;vSH*WGD&yRiMbdu6UV--R#H?$F>I=}AuDSy2kU zBu?Uc)OOOQViC_a`EmCr^y|E$cHvD)v5K9xtYBx=CrRS~RaoBDhoznHDkrqP6iX#P z3iBcrU?g54dNti%6)v!kiuMf$6EkicZkI>qKrI~l(V^5Hla;yva_O5=H>Dl#o43uU3$ zqO;uoLXDMn#Vaq+7^AfRwO@7ZzG5*Nm3~rIgt%i^ul>$ohmDdN+}h+wqN4V?8vo?H z&5}Yp{yo!-ncK_<#7?BN$7Fz*dyg;HfhTivoDdZ^E1fDGrAGB1Lv0ap3MsZJ@?W|N zMa7={1@w>F$vwW@p|;HZf#}?vvxg2eWuFKQjK>a>PJR-y}pS-W!3?zDDt!)^l`81xfx;AR;HE<>gx%X6# zb!>Fp1ss_;`?co_I6n4*CjT_rc+4|ZgWbx8ohH^n`Gvk@G2zod;|PjrnZNUVTLr_y zc3g+l>yvhYEmi+3Jt`O+R_Gn_h_WF~`(=pt>!Q*B>$*hql*4V>JnC%VQW^Yr*b2F?ruCQN8FG^ zmwC&EBCckvgjUUygt?kRB1|H?DIHP)0gM%$xea}R_EcqtuueTj%5kvjS4Jv6tf=h> z^>@ErCP-GMDgLIF-+BtBPZoOYmqSe2u%E550S@_K5;Vk5z9gpb!Ar_*x6gY;0x3;l z=-FHPYv0? zQ|^;1XOd;33+2aqJ?EV-qgN`3cZj@U#UJ9;tB$$?GQ0YSkM4EFWIX&@l@gm4KW&|p?{d{}TJ}o~9hCP)6SVKm zN+l~LG|@$8j(iXuH&0Mu!s7m3jc^3MSXxUhtL z@e?kF*q&shoC;4zd;wv#fBb4UEB`6q@WJdru(0lCj{1vNoeC}vX>(zk^R5}{pL7(F zD$Beha;;s^$V@5q(*P`3~T6qfkx0uEfH~2ESj%P*a_Fb%?hPCjJ zcrP(8{xiILWt09K#v+^!J_FXd(#e9CN9;8{;oZD!JwIEQ{Zm6yPCo$O)~w&X)idZ! zcOT%t&Vof!qQ?J9U-huptN>B9w4NmOJ-jX$(l>%Si*luoY;E?V0%#L#d@3`f)stpY;R7CY0R%%PRUWVyA{aTEpcDhPUR|{!i{X z7{2)fXk=_Gd7U%z4xLT8pi`Q#u7O_HMzfp2Bo0+6s@`NwlkAuHJTDs(8O%Z!Wl_}g zL)nKC$jUG`J^qVsGv{xo#zK=&%ronWZk%`7tDV1om!+i%8v_ha0-ro29|K^iV}SUZ zVzhO7R@T$f9xl6v9QVvY9pWh5W|ENiyYf+1MxR@@HWe>1&gLI?B|8Z&*j84kvx1y2 zjcHPrp!CcvS6KETfv;M${y(zbGAPcjYZeZ{gN5K8f)m^a5AGh^onXP;-3AEmZh_$L zI=H*b00Ru}o{#%|p8I?2oKv-{_S9T8HNWAle~QLq$D$%aszlg(gFg!O*9;jfmhOU?4l()iVI< zelYPn>(Umyy&K|dp+)ymf8HMC8X$d86B#yeJO?$eesM>PBaQu_X7>^9@sQ;uGw}zi zNLiRCM9&o{_^j!}uFsN48cbmi=ITWp+!?z{`qr|9F(Q5foClDPzVhX$_!1DWRW(iX1c( z{0NUak;uk)X|Kd8@D==Aac=GoZPNFcc0zLbvmLa5k^egBL6LXrw_GX3D?Fm)J9fm* zkA9MU*Un^;Vf36x`V;o%3jV`|R%^+xd4IIou{#hM)>7R7(jbF#ts0LV?@^io?5B>Y z-9isgut$)v`@$x!Jf5gz_C)_3%$Xz@Zv@lK!5$x%^!~40*~dZg6RL$dx6XW(#?b7m zYXcH0b5}gc-21LEamEzwFc-=V3}1Q2g$1pLa-QfxqzIz%NJ2MsRC$Qi55GqI$I@-T z9Z~(Aoafd2PKsL21`r}T{iM>}53vbv6#4G{aVIMQAVpoLfd>W4n)9YFr0aEkXFTPW zXy27T3F&eCBSXdoHqY*2dpsaxH!u41R|0>@1%qoyfG<4#3YgBGgpuX*M`EQE>{mrn zMesHv+19vC(mQJ^4+DYGtxxAe3qd;Y4V@*K`zcVh}QX#x939hFRAyR9YXbw!-ofe!As@e z2L?k(2eYfI^U=<?dcXCQl)9zh_oHI(9#Gn z?GLfeJy_jpuNl2L28$37>CX>+ksOV$av#${`8iX(1RrI5YvQK_6jqD+TD0ZD0ZrkW zll7A|i^*&buaC@7+^OpA*JJ$_++=k}CU4K?4l`_pqoDnqf7Xk`6SE0N!easHj`D)^ zw(SH}!x3bbdA;|k{Cvgzv zkjLg>Hgf#5#zz4WRJ{21Zn78!R!+Q}APZS+jBTj$+KD0|sv*hOdxc{C)$^|Hz) z7d+J|!vOO`oFr#lTIopKCR9QQW28K5S6i@1p48&`I@6e`uZP%{$B>Oa_xbPK#jdE? zrNKosmY8z`fg%LXSHT$9MT+(-sf1`EdZ^#>uX&#wYtz~6vu~fAi|d0%IMfIIbmlG| zwZ5Sm;^Dt3Vm=nx#h3B|AVbkTe8Ko61CB63Xaq6uAsr^6GyEe(esgmiDI*a)XCCcK zYelsQ@fe6;G`r_tE&|wHfDj;vGZceR`7PY>qQB&i zcjP-4#;w$|fy|wrP(knX1V`-0R9f3ZZNaE?n2uVEr0k?HvSq1;AEi!IVIlRey%Y^O z`3u)c7p4{?E*z{Pveci%%SvI^Dby}{Xs8tOoK<|deK}~j=E%j#8MA-dXdOxqyF3`; zKdKSdGm=s6UD0k_sNU8j5a}HV%lH_vR~SM&mEYOSD}qSen=WidJpsJ^88qZ0ALcCJ zmavE=%Q@&xEA-e{7hNwH2~)i!P-OAd8}M250D`djGGgSP@xx-H zz37E%q@|*Y_ln@}TdQd^z5&)F?{m6@X5z!ewyM%TOMm!bp^vFYmucbahxA{Me| z0H!MIP<<$bU^Vk@px%;=A@osR??BomU!^AO9+|ov&QunaAXak<^QrT7)SF_{DWNr5 zULYN(NH_D_e4$X9zv6eBz;tb1`^aDHpg+6jtvzBWIZGHxV^_#Dj^EHECt-GSogt zO2mH?etd&4&FaZOR~9^GV*VCr=P$JYB1M<6O)6DK2R@vj$?PZr5U@yA$}Jrh#rl0UQ%27VAe6qHP=57?Yo+8?Xt& z*ak!eZg>74wF%{a)Fw_+FRp*j*PGhpKB6R^V*|;Don-kL|1OH<2H9n#gHrW3wIpG} zJErFzd#3K_N$wkD$*jUr`#=?aPu*%Ggz0;i0x^fp-ByZX%+vxkrJv{HdR-E9o}Snz5Yx;4YKGORJhum&}RT}u+D z3B_QsvNm;+UR&m61 zxKYrFaw4E_>GmuYrM{8Qoj`k3Z<0kb-9Kva3_5Y zvw45s^n5d)on~PjAymKs`tY;y;mCt`AId>WxD}ZJQw6ubBwg6{XU$#6sS}h<^Fv)7 zwxoLX8*AyCjY2AK``#6@Rr$C(n_V{ZHJIPViW$*EHOAeXwy)mWKz#E@JQLAoNi{UrV9BPdDXM)u-3wwT_)>m~dqfkw* zTj_E_lm6oSvm^T{*1a}eHGqAMTu8&@_sq)% z#3OWb`zVu?@ID&F*2xQ$z(GX|Ik0dun<)Hwv5-=?g{*8Rc35Z`MyJI_GSO=#zvsaM z42fN*lwXZoB$2A(_KC@2aP&*}B+kq2L`|s#%<)XOI&vg(Vk9CxvV#ru&DqP5Cr|eg zLP8N+Jq0QtaoJz7ledpuC)`kzIrlFT0f}KJ!IB*cu@JIriRR#UMQF0qo_3=-NkZX3SUGiEPT3uiE5cC*i_F~W=pq3d)~fp z+J2kO`qymepPlFI;}W;TWE=T1*2;e$-1kbp%h?Bce#8CL?Cf9k>Dl7w1Kv*Yd-BB% zN)*NmnOJ();B;=ULiFrTpqg(tjctpNUUt=FmLd-T?xse#e;|!6A{e*@j^0!Pw5)!xa6sT8i?M6h&N*3*A%6Jm` z%ZX6Bs@8dyYs~DjZ-52c3ypPt?va-vhap5=!4$WhwoA}>mjfAf({c>B3`UU3X? z>F*F1jjTF+wHX3UUR)R(@SaWhZAmh;YsO^H@nZmcaeM&Yf=iK3Q3F@ebGEDw%A>!3 z@bEv`!$regHJWl7Bs6=g@6=`x7n8I{(F@KsUdcaDGEYrNzSot)tXfxF0`U6SzXX31 z5CZu#zfRgmAxyH8#rWN0+%6u@gnMK>KCtYVWxEL`Ng3i%F?&2Y99~%1t^NXH8_4gW zg{st3#4i3kwhDC3!awmouqaJhWRE-Y-?`zmUP(CR#A>{oGGvyzM&7M55I1JM-};?xfF+ zmhe03*g4mDw`hvNt2$P1!A0>9wV!X!#0X@{ntOn34?J%<^1JJ67TQQV!Z|qs;DKr@ zJaYL2`i%#_4jac0ZQ*Hj-c3jMk6k-(lVd`XVhb!&&{$N0uFDEiJt&02HmE0ufa9}KsiCeRB#+{dUj`6WBYhYFh@%@I0!HXaaZ@T2rC$-Yc7m6qC zxvt@1-Q&%x@yAHj=H0?lE^)fj#oFOagQs>c|A5xpAC;89b{oi^o|NrQ{ zu?QxL{|dod(}C~&sw}U?WjQrXw?qQfDMZ3PEIPDLVDmAmd*n&{&<)-eL7~MHYd_#B z#mkrq&BB$7lsLN|Zlidn8b~Dm*}}?@aKp*`arU=C{Eq~lNov< z7R{|0D9Xnr9+8nZPw#Xml<@N1>>pX3Mb7&~AHkZlaq|8z6XDwlVRo@-VnBPw!mLPc zV^)lW!XvoAE3t$~>}J}e*Hz;a8WN_0+)>NOrXH{4QN2}Jq6)`32dAC9Kzr%31cC4K z25d8bhFTs9Jak543@$Wy7Wrv51W2c$4p(3DXMbPhoq4P=&NVoh+NphAp|7rc2gk;A zs%)?Go+xDYlMNtPc^fj|tc`cc%5Wr~J^X}i5U`vrdnvCXK5N>me=fm4PQ<$<0QOjM z&*3$&-sqbEC&6_udP{l{Y7U$}1mc<&Y}Qb{uP=9<&W@E*MX5M9QLa>8o8lRHH=#zp z;G0i+pD=RN(m0%f3Ff*5KPbKM3|{5=$vQi!)DH$_R~aF5F(u%4t~U4>D(r)g*mFO{9^aJK9mdQZmZo$!^)*$R&9- zC43?kW*HiN_pL@8cSz;VUqDVbT+`G*Et+M0s#;i5F9j*)9Z@7|MEAGsJ^S3C^f7ta zwcs*Zax_Yv+pln@n#RAs$K#sTo=mJJs43aH87d{{h%}Z)TZAlET@Qx~%|=qoJGO4B zz?b}tNWzAG*^pp){wBPP3{C}(fjyr4OM`C8)wkJe(b- z-&P|$zHLd8Lcf=yHrY@nuakQFQ;B5Y%t5Pw=>w@@`TY)z>Neo5fc@3$*eqS!3J~Nd zxqMY=AN!<&H`!F5c?Q|*w~P@Ih-xFo{@eQaZcP%LcV9UH z#8IjOt9=yAJew%Mp+Zs2o_ZIuL+?v;obY$IaG~ z{_aTeIKk25mp7B@&pzYF0@4-1KjJ@hR-m!M)e0g4^J~dutfkg1rN*nfMaqC^%E1gWS_cV}Oy!&v0L$I-ves}R8|w**Hw z&bS7S1f;>EjOK9GE+0Qloqar+KEtLz{pZ44<r#xjainV{wDTQ2PHg8C3KaDS(2RBE+i|A^nI5af$XXsE&wfbRFlUDaW9jYK^qZmBX{k_BZ-r}>M1)SwTun+#u*KDPY9 zPUMCkA>s+?Ol({_PehdB$&Y%JSqM1Kb@k34 zVX7@HuUI-v7ZG})_@32zV#$*^4r!f>U8?7%9j%&S_S<2Nrl6KN2;f87DQKT=t}Xn| zfH=8!zUSZ)(eLT3+?WgcA^&F(8eS~HggczbREqN>Nkyr$IfIgN0H4lWIb%gkKlJ7} zFOyFLDbZ(|^TVDJNF}cC!McW>WdzcU9Y7( zZ3j2YeT=9N*t#k zp=RHA{r`k|n;(}HqqFTPY9V$1nhG5V3+McV%)kmAPJD;=rO(7hVa~l;{9#2$4P_O7 zD1&0Fw{(6c)u8O(+8MrnDCR|`CO~V-85`^!7Zrk$mSi{x`c(QneysF^dh?%a)1rqC z%fy$&Z@lf_5$pt&_px9LCT}7hh(lJgX@}xVKwpswI_+$G6BHM$++{c17kRKZUZnrB zZ;u=D3-q?c53raEbLG!@JN!j7R-eM*4Jv*M;XUakd`r>XMvFmqo6vX(E#JYL5Lo0# ze<9L#(6Qp!ZgvApQHdyNpbQc+`<<|x*em*LRw)YG?dj(3W|oG?JOOid16=!2yC)5NVKFJUivMLMUeHFNSlG=zb>8pu#yJ>s$Q~7*p z#H{*ICN1H+y;t>}(u5FN7JqJJn}gB*T?Ic3WvR8w!B}mwLg{>dA9*?x8-BjH?6l_B z;{C_FJ_zi^KEFn8_GvjWU{5PhqqIe?`eYSD(@3$Yjd6D}Yp(eM>hzYg!RWlFX2rR| zB{bS_hDPsZQQW_`ln*ih%K8KAm$8-FPgCJ347&15`D~pir?7vD1;G%NzYoRE9Hq9g zQo%(&OITyxle<-7)ZmJ&V9Vi?>1a+^KCXhR3)EY+ozH^T!YQ#TkD#!RCgG0+(UUss z6p`aCOeLj!?^NkFLqZUYH_8lkXZbqW&S!WfY&)_zNo3wm%`v{qbGfC#mH)~ zBELk8?O{@<&bE3v@o>!bqWd{IQPpB~fN$|+Dx!xW&dHq%PS>TODK9>aZ&S0X&@dY- zo{%U2eQcmgAsc0yR$!e5`pi7`6tN38Z@Rh=2hgU>oo&8*0^lFquvcMtOx!(N#a?O@ z2#DAF-IS;iEP{EyKXww67V1x3zepDoC+2!iAAibNkDns;)c{NPl}#Bl|I0D_XNR!= zLw-N-^Lp9+`@kR-X|5PG1-;o6^X;DWw}k~x8;Yu1NK2(vm?jRtIxAre&95AwzTA;~ zW3nlF#S8w%cCf=N{h^1x@S>6mgJ6+L!@mR>1@q)d6;Z46Y}*7yz6EzTwHBonTUu&B zlhcdMJ>=Orv5@5ZGKzPDDU>JFi^~APcr2!ZYmmQwp$)gbn`ho+g9W~be~CI@Dz?)W z?EkoRZE8KOZ${arfU)8_l`T$i-^LE(L6qLAIO+tI+Nrd9Yy^Mp7!bmD`7!O}LYOI0 zHshkLKD`L~T0?THm%)J6yBc(_dTPl<|F)I5LwN?Xjj5iV4kl)xm1&e&pScW2@k?-1 zLqdVe!cTh01VN)UB*rovqOKp*(X=!)DY*f7oyd9! z2B@5X6}?iXh%lmmqpNuvV$a)!RN@h11|8``R6kkvQRhpa>m8JkEt1(sCX=49YaVFf z42o>9jP#9AaKV8-fBP&dGx|PpQz81zG z&iCKkbZh8TR{hj{_SrSxuYySpk}n;6Ld3H8&#euh8pP{hG+ zD#fjS4drsY0Pcv3qcUHK{TiATFpB0EF~!jbQFJA+1yGR6^~NMwypF^-to}{1;Bm2a z9uOAcrJ2ovQr*YB@S*C}JSXve6~+s|q-x%UD-uj#k3r4}M~mNNOz#_JTKm<@C*s>= zd6^eHBa%&8!}C4+wB~ob=D5kez;>`V_$Km>COB%*+Rt%9r-k|t(qA6OwX~|M8egdJ zX5^LDhttA+rzCF=J@GC4&tb55R}aojvX^7%Zc0@JTRWML_s~{uh^j}r5V7*zhnXm$ zJyLGUt2W-3HsQQ8gwE4P`jacGmm#`$pq~Xlv>gKUAKJtnSc5wzA^Z0}f__y)qZE{} zNOAlxjdb44)Pisk(a<|U)pFV&(!uB!U@k;|IVwMvi%=drr&%E51rRe#Q)EEWD)2VG zJi<+U?*gX*Ll$R%5qe*9`V3;b{Mi?sQ7d^lvkS|7r7$lp3@7%VwdExz{1$IBh?RT( z<@5Wu#V!1PZK}p$AQYMF(<4)GY>HPfZ94kA4VL*O-A_SGQ=+fb(6<1VV8c4lN{bbH zk{#JgJ@@xxy?%s*B=^tyc7#Qyz>SlKp9RGb`HX{-Y8dy8Y(Kjr@$o^T4|W-(`N{8S z({{H=9BRrq1ZH4$3Vzv5g;Lv0B+An7)3t=?U);KOA=kifbaL8(4iT@C!dCMFyd2~x zcTzvx7$?}tx65;LPk#(JwNW~yF<$6ime~-rD{uQo-zxXgVrqX5bdhng#+aOwV&^D+ zs@YsjD>J6|xc3B6vo7v+|KUQf0h;(c6;tecbCDUn+0;Z2CfJ9t%Z+hE)?pPVk_vE? zK<0)G->@EPVR3}0UzF2;LQNb!mdGAYsJx{OlIqFm$m_D-^|UCCWdG{r`{M$sN759dOY7yM18PP{A$a# zKwySc(?ZWquXmC-PH761Q+v_dC;SY%gQ-i0ql%~*Ib9QstUTGKaw8RgX1uW=fZty~ zh0vPGeI4c_eV2~Km$Zx+u_m`wD~m_{WaJ!1BgVaykiR(IX;Od5Y4V7?>{FqiZj&2} zKF*;p@7#M~ypcwmjz8Ui1k%SKUtVpqeqJ}=VYNsg?_A^0$TW^0EigypAngJ)JE8j!IIQeK>P6WqqTR>V9{iSCx`V z>qsXEwuwGP=;CNn<*4B;&F`z?Hi`Xvg4W3wJ)WIDuE*=d)Vz$(m=Nr(6^(98D%%pv zsT(HXC?ZQ=jQWxnne>*w!BL|X4xPx|4?UdD3S*D@b(QnWvl*}H)^{pD;PDG9Jo1}Sn;JnSK>>u3kmNsZYeh!4k`Lxzf7C!8pIc;OL zI1BXq1D}Yl#4vz+wuhTfG_Ro^$MLNQ;8m+0=9M8ALYMr(4@f0KL>U9%GqUh<(SJ{r zFkB=1`y2fft5z{@(XHCb6t=%!oCr` zI(Q64Lp8!x8O;&Glja0C{*D5C3vz-Xq~F-Th#^jnInS3NFT(P#f216 z{}8yWgo?qt@F@pZtbBTEMQ3MPSh*x46Y>tydHsxiEnlb$B6_I~B6u`1<`z#Y$f#2g z;u+XBsg}Y045kSK^0s-%$XZA7-moF8fXGTq^hQp#H!Qq!{*Mr)^cx7`J%3~AdFYHw8f=gtlsWH26rS~ zDWE4zgBA~Yi_V^`ujYgazJ`bf)dV9px{ic7_(j~h#A!&Tcazl7eL!7U{R{V-S7~@M z(Urnu4f6X7vg;zD#ezj%vkd6BmPxI5L9ou}gEuK-C2ob<(%Rw%E4?~A4L_$Nn?;wu z_r~F5{kJ^56)D3P%qgq_sJ%jBJW{wnQh2wkf`5(_gdMe!&OZjvTZNy?5R~W8MtLiD z30~D2AO&3qZ7xwB=2-kz+*+j>e~^BN>r6AwsT>TI=cjAE#46eUm1BITSfOFl--95z zpT+hoj5I976F-Iw*P@HAdVKAEx}z=|0SnnoTj$-bsY`-gnu*(R3EI{W@A{XUgY~ez zfopu

IwwC$1Tc&~cGV{Y39{qg%NCw>zJxFn^x0V&c!vFL>daG&iO&_j${{q1feY zlWo+LsE=4BMp+10X&dr^PUi}zQsqEz$G){`FeM%ODBl>4$j^twnx-1UTfx^%RVcwi zdDWHsOkzfmCJHMCUEMk*#KipUY8(0U{QRQzE)r#*a$j^i4GKyf))kjblk{k*d^3gc zH3ezpITm91!cejedfNAcrvzpuNiVmA_AfKht1Fcls;9e|Yv9M~9nWzw_Ol8d$M!7Z z2b!i%eXQOwmIpwRVBWcC0_w+xIqrz$VJuTF`)44exw0n5L)<33jL+2+>+9M_vITob z>Dgz6={Of3{4SluMW>vSBZ!NUua7YCftGfIeuWLP10krtHuIOdE8#J*l_)D~MEg!`*(`3bll>%Gh;JdVwvWlR%4ej0b&Z_m|P#7Q*D>yXKo~XCY{{<#4QG(zJ0caQhwAIew|Y( z6D1encNvcYF71NpFZ`KZO`l!c3Bh+U*)d7)!W>v>E?&)lfsM4YiLg*bt&#cuzJ!jF zKT2#bV%RVDI4;@zjZq9QKHX{PEqJ&Zo9bN(yG!BV0Bq}@4OVvZi{kAK@C#g);vF^O zRy6px7Hztc)c*dc+ z?V>BI9+bHLQ@FU_y?-q`d|mcDg)Alan+A1rVOJ0KWkupUsjFgmk?^Q2?DG(v1drp~ zZ&NF$EO7+V+51jyaL=p8`c^xj2=SK+6z&AnXzlp6QWNsqM^COg+-ft|6d@d(M5hw* z5My(BJtCyoN20kDW4^sTU>ztz?Na%^JjBlwEZ+~hLNkk0U-A}S3JbPI$FJh#W`SuFQA`7zYP}$LMb=NMW6YI=|@?nag@uIOib# z_76m)%t6%|k4|-EN~l{+0~~wt>ZCG55;hz=9v}OER*-dyEVQCm4hsPwvqY*J=U5M7 zxkXS|dSW+Ptu!Lv=$zXEv$L(@9Tvk*CZvgzBgOF$C_F331q;Y`bsI6{p=h1qw&Ic| zkcm+Yd2W3&5=eKvB+c;;j-9}`^%>K&$`0|$tkfueWAi7D4F%0maPAHjp_=2y_wT`Ip26Q;^xBJoINFuz9xDQ?UUJP+gW%JY`xdgei?!$iWtua`rc#E%K z%lv-jA|h0rfr&NfOSFH(bV70X|cC%%#Jbg)&s!^7qJL`m@VS8XcGtc;C)*4H&f_LuZ z+aG(2i1Fu4cFL4oSph&O`7=P2%cD)kqloQdTHuAW^4wJ|`Y^?Nxh~jwyE$$C>b)>~ zmBm7c)gssBw`|gqOVsu5gwa85fy?(ajV5Vw0gv;uYb9YLv#^DKkt{qwH(U^>WJHg2 zOoN;S4JhRzWX$NLlif?toL?IABg}u7rXrlPiL=0Xs3Ye8eAWEz<9Q6sYV)=No-0+g z^fbn)>QaR5e1QC~S7JE3ONLD(2Z2qs8Z@j`-GFR+-n9NxPC;8TZpg=mr*DkF75VYd zA;b)duesywCdnFFoNe^J|p_6 zj**WrVBX?hX7S7sxftI&TiHA*y+ObEbRbXlk| zvV`JrgVn*~E?KULfYwar(ZQWXmGs@1{^Ox`QO^QS43EBS-&*!#%{{{kv8Qw74|N`* zB)@%iU>=Q9zd)U3#2+cAY*3_yT=T7#yv&s9gCEooF1J{9CSjX~ZwN99%K!BOAYW)R zAoKQN)+e)JgtPY-3GU}4Xlk%k@Aj?iyN7q+yT4AhOGT5!GGXN8VOeVjQ&cn?}MaMwfwj0dg=afXD^ig3{Yb zHTUoc18q1C&zcVBs3v)+?7tdEHzR{h(2T8;70S5P=y$fbTic#!(H?+yZ5x43lV;Q2 ztiA3=z|4g~tOBr<&u`w2Wf$kI1l|Hp)>L3qj6f$KFb_93vYa?uD3B8{6^yK21M^o+f1!qk&MIhX*fgnO83)J1%u84siF1p>8$w z%ALnQ{TvM#ftj{{GucUnWoOY4eyNSg*$xJLtdVSe`+FSAGypyrb0WsL z*ZO3?+H7ucHB1m0-x@B}L2IGE5LdIa&;ph#m0rtpcmGT7$5ouTi=g)99JXUfuS+`) zcjAZ@k3|M2P_HOhlP9;onH)fUZ6rrtc3{Gc3&syh8Vn+F)&q`pjJN)=kLPw|B|=*9 zS`XVf$;RT?;(HNh_7j=VSpXUKbVGWiBP}RYbFmh-oR$_LW-Nq7R#0 zW=U&OTBP7>0BsGeBPTc>hnZdUCJTbNFKYyO-431n)pD&R&#N>sXCb2}hhc)PuclH; zI)U%^5D^w7aliR*|c@mxwVAWmXRS6G%YE2$lk81vi1icvd z^P9Og3)RR2EXr31|Md5X@0aLW#64-`EE+tzt+CkwlZ(!4#O#B%=5&VzIsAxK?6%kh z_Hnf6w*_c(&@G-n$)A9wYelPpFs#Fgc(wY|tz2*6i>_6Asd}d_rH8vi_&g$x;jZ*{^DQ!O!89sN|temo_iN-4* z&;RnF=8gqTDa>23Udp=V_4{XGI39ZxQzS5i%JnpkaX%-Y<0*DjG6LKDFUDA&zBpdD z`aTj7ZNL9p6GJsCQuJ4i%(-BBRfO>+bnLC|TyU5bO}=(=;&jH^IesAS&A24Z;C4WK zz~D{Utm3$4MI371Dh$RzZ~!E~_;3Yo5h_S}*t!9Lqku@y0v=V_y#E;s0NvJ_g)K2`ffd4r(bY{09rJ za>Dq184&_Z#9;aw3ve0Rm@=4g?%tPKa~K%}*w%E#9lh)GeG(HC5)kAk*q`h)L>V+* z1&z;~;qD=zJEC-4V-fM4Pb!{Z5D{(QdQ8%Pg$S6V7L-0IG=2v}S`L0Lu*<23&^(bF zh1Q%SMF`b}n!JjA1Evr z_tb9m+4r92HYUinQ2uhlmt#lRj=Fj?S>9$cRFk=byWX+e>9l9{?<);IVDr^f1Qhik zUH2P`}Ek-#fFR{dT_c-npjGw*VMSiTwYn{O=Ifk^GRWnie$hr>^ z#SDy(xXxiwW-@cg;ZR?h`7E6~Z(@=o3Z?+8ISq;@@9R96`HDqB`X5~SMHQ}th{Q1Q z%2pJ8!mPOcS-hhm)w+UtaFTI^n2SSsCK0UhC!?wC!E70pA4tJU}hlPMmPHZ#d= zgOk&_RK$*JT!K;>yDg97zy~Gc2~BobqM*H1$oQhukKhEygzc#bo-+Je1F|n7{H*EB zQCwceiHeb@hJdYoOPLC20QaKpM;YtxYWc&jj-i zBLiQ}LBs7CN;@Falbte@YI|$qgKckHTW(H6rlok8^A5OycTJ3v+2}-TwE9>8qkd-N zE%&vS{j25qyxqLmKJWe{Dqaq@JO;-fnL9iOW4+J?xm4HPsQB*WDkRFIzJpo@f!>{j z{9^w&tHB@lM4(i0RXv`meXZP4PW71r)+=Yhij<_qgGmKdMbVfK3-&X!Km={EQI_0c ztb9&|J4lr_<(5k`eg`WB^a?t=l(0rI$#RF8%SfU%9GRtP8Qd8*Fmkvk>(}*7CYb@Z za9Bz4?%;bi0vKm!Wzvy^`OX|1bq_ zA<;g^dH7w-tMADJW9XFz+0sQRN-RWYIM-iy2R+O>SA~a3-AR6QlFME%!QyDw#;?sN@h7K2|bklQIf#I1sInX;6 zMwG1m*ofdqHinQgRsLykim~QP%als6TvXx}jri9|wK(j50piC17l}7sesMXNmgIjh zW57v{u1f^WxSd?KGE5h_u zt$~ZlX{h?0^4)$dS0&(0%i2(I;|YPSx4Xy;R|NUEcHv z?75Iv;SJai-{jG->Z5%GNZM0T6F;Cn(4axWUN=Ne@GU*_H%vQ#6t}6EA+TDzCrXAP zu(&)b<}sOZXnfHIv{Y`3eMsxBr_3ZXv5ezr%?}Ypgv=$rK0g&v+6QSQG5Hrf^ejL( z8NgMwbqjrht`yKeVHu#sPDTrh(lR6r=zp^kF^HoNFH*9@KA0`vy+|iXux+?Qi}+1OiAw1);1c2hgQ!5%QM8F@9BQvbzsmf^ms& zik0*q(k_35rHAKRehMoD=Z#%bW)=u+msyXYli-71KAo(9%GIjO?VxNB$|lVr=a5-8 zH74Dr2fJOzhG+*UfuDU7PRhzx4zu#==7@}`&;7m^Nv`;#?}+_w))|D(3FeQBAVCgT zI*bVdwQJpA%#sQ|D`!C#q{F)kR%p87w-p?pzI?d8fILP1^(Dx}?IRBw-sa{Rm((44 zi}1NBKxwA?P}mL_SvHMnnZkrwq1}Ps=8tFg9Oq)1x_4>`2l$@1UgGVnCc16`i zb$5>u*R|r4x2e~Q&=DjS0m4JW0pm%)@EJHPxoa>`UQCNmTpByO@SoI99-pb6wmJ9-OT+1 zF+cRW4{ID8WG=fPJx8}g?yy`u;x}Pzoy5K>MtowCZ6o^!b>LbxT@%BZ49#VwC-(l; zcd*RB&m*uT^Ue}AnTL+R{+~&Wu`sJ?js$xDZ64SdwpQ8^CeZhYRuutOERS;uOS_?+ z$C_ej(8YeMOGI2YaNf!uEt*dX`-EmtKF!v$DaiL&Odwkc`2NY_dF02GtUldrcKi4{ zHuS_J-h^>F)nUpEM*!BfMY9^= z)p+B4;oz|4I_sb{%nvJ6%om?A_U?2OQvDbh&$&)gFjq8SG%wbp#`e>chIS0|474Sc z6RZW7;G5+?sDuisR1}9s5XqH+$2ropTUb{?{tHSeh>ip=;>Bgv{p9IKHwl2m=wSI~ zQqx3l|Aj4zpE1gB4XVAcv&s9em&Mn~-w9)`csecs(!tpdf#?|fY`{Tng2^Ilk3LE$ ziyKXncB`%#scq8k4cF$PNGohcQ1fZDQL6YZAA>o<* zLlU@n8aQY|2=f)ve6O}2aYwcbmpp4?BZ_dXpTkj>JlqiYuv{h?q*xJQw2@XdSkP&8 z9+?28rji36crL#`)CHB#2X3(B_>f|*vKMw{wd0PDO^(q0N%6Ew8cFQ@vK}-ZGH``q zT=a`zK$&qULb5}pfQkJ7;2ZxoHgr0w{FiV1Qtg9PJ&xcP`EHe=j;xI}K+t!EkejaB z4<>@bRB%{az%`iE+vtpn$~!~=&Zp>AQ=EY4Gmk_Bo8y4uwX%wXYc2XUtNw^uas7A+ zOWsCdG6FflV*O`8a$G$k8F3Ht3(pV&_>7@@SSz@Y>ADYr&G{Er)~v=OM^Z(GFGxaa za{TcS(eo87_-8dqHeXwwn1~SJl=v>O$;z2Ty%*6qubeo4M`+_8iOlS(ln1{Umfob( zh-2^6E7@)4v)%pnC1w3aH;UV1LpLCA)PPN1$f9T=x{#J`bz>d%MV}YdgCysb}pQ&@mYms+!jGk#GMz22xYL?MsuuJou z9oPL6;?py5Whk9qK2L774QSgg39&%_r^YSUtX6O#uQBHbYo`ttrK+e{s7_h`&% zh|5sTajQ71&0*^Z{eQWTe{?ZVid-6r#not9vVRfdJE83jA+KOt5*w{eSnN|8TIxhLZUW8^ z28&hq6r1V-ioen2Z6fF>GCk5fJ`DS=V3kVS}i)<-MmBUW7)`JcCcum?-&pQD8> zO$8{XEakqpcz&q+13}w#zwBEJtIsFv$;gc7zc(W@Ro`c8v$Rhl{43|sOC)jNX11|0 zN<-V7BRNt^NTj_Kk?j#F7#@xSI1_OTm85yB3E=TK%3E;zS^2s}rl>a8I{v`Y_GqUJ zw-c~t1VOwSE)+4xA*402x3_T`-L}a(Cc{gP>CMq)1*bBPRX!oxvz~j3wC0QXjqA*@ z=fpYPj38YYXBo;ntgF#P2NkpwLAD^eHcVZA-U=2~b$Ar3c1P^z1cNWjiLL@?h34zN zyMbw=(Y+TGHrnRbaY0%3;*xwGgY9Ot3CYr)!21rY5HX%0YKy=Qrk@=)S+y zp)Vo7?~O+E-t}xSeBn*#>{(s7N)6wO7cEprWoC|=?LmbteV%n|ZusbI^eOBqfNjOL zZ`BTH_eSu`6Lc^clYWV{axAW)WI9=JjC8oWkm!3?C}e~WX6>4rY=)uKK;dOo6GY;h zgz?ecBt_(Ec2H`ZiggG*NBEtQJp6&EQ#KZ!eunI(U0V?GuHL)}P-JyKimngDpW^?f zo*n-Uzvior-q*gO#YB^L%!-3#4v~tBIro437b5>Yw6d>rG#STA!8cxeL_hz**<|HK zBs{yd3$lw^e9?jh)?Q(n!dfGx{eSGeXH=8h+BUjuU;|V{M4E^g5di@q^r$F^iijvh zNsLJUr)je#-2esAE`5fnS$h+%P=2V7R#|GEKOop(-56&#Dc9W}lZ(v+D5K^BA7Eri zIfuSs=ap5%oesxht@3$QMKA31T!`AEKiF)x2}=3_%#zQciWfs)E*<2R7|x$)4g*RqbDI?gX*_UDGfJZ0acr17^~reW ziV~TBS9h+K78n4T3Q0TC)Fpz>_zi#}E}g;e)OVeE7X2;nsULpiw}&_9n_VLZEKh#m z^_M#jiJxI%XAS8}8=GjCN(AFm)FMxI_gL(z25dp?GiOv2WoPytRuXH2pyY;g8g3h} ziZ~)$PurQ7a>OV!{B)N<&ZM|41H2VADI$}{THYWAWt>@Sf2!x&Wwda*9|Oq>=4}LL zVbnI%C5DEvhSfG2Gh9~hrdr=*3eFaV!OHqh$exJa9cu>a=%j7TT@l+T_VQlxvIB3P zQ|QH<0>zte>AiTk;mVaA;S&vy%#_zqb28R=wVZR=ds726*E*@k9(&M@pIwg>=No0` zAzO=j67;1bRDcv@!6V|F5U?qj%cd;8u!v=4;Ty`s3P&7NZUb0DV(d=3%q}%79Kh#n z->q)NJyQ{h^uJotP{^Tb^rW>m4Dq&&n722;d%+F)CG?>j+T!NWwhzoDbs{|ZE51$! zTP_m4^kc}=cCZ=~_@;ARSA01Mctc#tn*6Nz*_4|DUyAdOrEfMhlAP{8w}J%yo6Ek^ zPx5yyTjPv{xb$Pqu`S7w#v8#QT$^+1+j{1-%H5iKPVx%^ku>kHsTc2st(XOrWF1%~ z3%(fKqM;JR^5vhIFBUd)(c&+>9<5)UlM#6^caSa2VdqBM>Q0BlBTPTn;?9z_)CELV zW@q}D=_CQ7q{x)kRIuWx%goYMZV#kjal?+9)VBM@P2bLw^91qtM%R4+tws{Ms%b%h zso8d*dfN1YZ?NeniByfq#0#l5wQ1AaX@y1{7CTaP;uss2ZJMR{Z_m?aTVZ6+`HCiY z>5KWON)@?>7dJa|=lhQh1jSk9a#5`C9Ud4#OfNgdE>dv->VqCB5{Fjb)3iXD9pW4yesEb579; z?m`awqTiNlovtut?Su)gobO5YSpx#R7V!en6cJYCde| z**-@0h?tsL`4WAg_C&*j1f4CsExYNs_F6MbylZn;#@>wiJP8P5!4q4ScckXWp(W31 zog9;{ggl+@3e0NqAH%OEul20QV3MCthO|6#W~+rwJ5;MH<*djF7g^0ju;XP9AUc(( z!M*3wxAM0a#}owHMl9h!FZs7DeT_er+5I>vU-Mk4fW5&>h0|+6NR*C9amHynrC9Ic~i5yK+G~FTmuy+-E6v0f_{#;OA3*e2C|)NdjY910_;cexFk# z#4Xc-Q6C^O#^AzU^v~^#0*n|8akY@MIaNFSg|wWOs&ist!1a9K($uM!sV*<>AcvN2 zG^cq}+WO87@zBL(t0$knmS#6H@{j9VT>pAC{rq+}QNjI2(!FcpzL^b+LfK@8+$`9U z``RWu+k;%+%a0$8fl27Jd_Y@hhmXGguJB{QaIJCU{q1|EucQ&3Lt70@NjU6z)L%6GmLI&od4ze<^0uA-HJ`7p zw^oQsn~k=SEq5gOY{O_BIvj_-?8t8gFlOGrKQNa#MnB&?uGFRJ=wA9;WpaS}IwUM1_M*)kYz&gC!MjP>rPWN0(ti{Nu@v(D$;V^xbe-=%d z^nSfqHu%#=jyG;0lyKrv45)u96yJVMwJ3X;@^CNBV$`?cl!-5OyTL)pfSnPlV_Za+ zmwtFT>UF_LDynQeW^@)ZW5>k2ZGd zn3-@RoEi$l*@kT8l;`s+ovf`I5TS=2td=*6YS z77dFfUSA7#auJK{vFHRJq_A%iX}SrRcp;vD2zF0Jt=sDYgF5eTfgZ zoBO7hZ%F>mnLanX>b~k6O7G>PhEJE$Ed^>*wIT)21h52@A-vd|tv2Syq?&pKzGtw4 z9)T{y%VPVTvl|6s;yBKD>8X+r)_ZY|?Jz#|u#n<8g*DjMdyIy@t{PQSX>nqll$lyhgx zM!~=Tyc6wEL(gs3k=uiv5wzD)GJLnfank>a0E7B<3C*QV;|EhKhqlOw8E!emZ5P@=QZce8jl=&jFw&GjzPAN_BiXq z?6{JXeK+Wj4km6myYLn{fua-t69g2D*llerwsU!Nh>~iBZ zv4;gwTa$%1gX^0B!>`1yl3+vwrIi_O=)0BIEdUZ&oerF>8W8{JY6@sVJLibLswU zzxSsHn&Tj?jYyr5Bip-lTFIGj{st=!eWLx^YI8SK5ps8UKDns=jdo>nxXsJXgXz$p zb0<_~S4-05wLn$Tfn9t7Q!1QGc10q|x}Wp~;sKz`svSx-$bNg8=0=s9iqwVl95?eM3I>kgzqE?=@avFKkdr(RPQz}A8mP~ zqst*yccTR79d9PxBOhK}0m!pp{Jy*tuZ6$a2>5EQmzg#$3<&NBNZ5Y!{s?*H zT|(91c9UQ6!s1TM`Z*DXfsC-3kJ=$D$;@qhx#k{l0<+lcnSc%9LNSBsJA75r3g}0F z1NkEj#y8H7WV1-9C?`5ru%Dr|NSzc;ZceBvuab7wSc`s;w>fupK;~BKgo<{%O}&W_ zfvb?irK&GlNPVrdtJeLX@c|o4zC|f*J&PcOo@#bWUbQShdanDhQ}&&o)0MBQ-L)&| zY7*zuAFh(-!&OmIAHg2#Qw3PkM4XdLw{X_;{DlS$qd8)qrwQw9{rlN&yXB;%_0;;U z@%`4EU5+F6R;bg_73Qk04JpBqi7l?G>#1KVPdBZe`ypRcpHlTm6{ehJ8&te-VzV-Z z<432J$^o8D`Tn$n8(fx26EK#^Wo6T-9|DHi(4|OP%GTX(ERX<^%de5Tbh`3fbF(so z@e#`#HZR%QDaw}(zQ3^MQgtK!$uTxm%oI$q5WF352q{o-eJ4-S^eqkAynv{T+)P3P zE}|5y&zSgeZJGzTJB?hY`M<2#UE7YAH9Z~PR|2?ak^>e~w%ljPOx>;BLDqGCIDrkJ zX-W7ZZPHGomhK(8caM+Pa7DRGdZ>z^C{X(%IzY_}fPbfVym-h(Xso!@RQ{M`|)`O6hHUwc*T4`AcdE2kJ7`vG#ddXn0wRbDfhK#jA* z&Q|#Fp!CJ=?!1QOGvC@X{ zldIv!35yC(9IF!ffm%g@L2O@YZ>uGp&g8##?(O#vrX*g6jBYP`b+w}_nTAPF#q68e zlikLrm0N@vBEX&E=BWrR5H&9)r?ylj{rW?E5rxftKF#&1HM{D5bLcUdxYWeovM>ho z5ogMt&fTigd3QGm8Ys5H)0;Kl>swkLVs*Bx$}~YhMLrXB_NvSABk$Jlg9gtnm0|Q< z*zqX6iSG2G)@_yfh~zIk=ics_6mf;VZCCmv!z>toD0Y~dnSTDhYouS7#`PNoQ1|#H zwk3XT-Od(JSF=SCTXB+inbntNj})D+ulz3G*W8v`Z6{zw2T9L%H+D|BnIq4%#usqx zjdrdV2YU;CvtLv(=m=CmT@@!j|AW z{*ZX$v^(b~p;T>3)R{7wdZe%fLGINbUE1I<=soY72|6M<}Jh-}WIojV-Ny0mTK_(v1+y;3Iy zOlWizT4G&*dpe^m!2V;>AJ*0HR#(B%h319oPwXPXzvK2U4HwOL4^_~9YOml{4p0D< zJpI61)J7x6Q{w(V333noW9`-~lArW!nmELI#%6Mc!A?29)5;uEz9S5juGF!>@f0V; z4rPjOJH~e5N&uIapS%$~ssxQn7kBm6Y;)qY9|Y{!MN!X(39bMX?ip3yj$l|zdxmN; zuiO^)EF8c45F$Eba#HRggG$U6lN;} zas^SnX`Yu~`I@}-jyO=2bR^2OA=nt-+pJ9`19|ND5qluJ=)kJhr&JB@I!N@|y zY_0t``%-z@QIkogkP9b>RtJr@H*jW_QorfpqqQYsxv-j}UrRqEjsQRlvW zJE7F^@qFuNw3=co*2t%Dc>))C%CzD*20$tFPFp|v28sBiA^F>kxu_sfB zV(ReCTvrm}@kEC2?avLzi7QZJFI9n;0~2*I8m$+#tbAG%?5cje-VXjLKPJ#59O1tv z(e3R%Y6!CAleK9RraoE|vh23_kuoE&!jkUNb?veBysw1t6m0LyIsv#w=hiy|iP`b@ zTg!!FB3Fb7?-nxMV(sr(qP3XU*F)O;r7^AEss}!*_*)nV@wnJ8?1lB8f!Wuci1e@( z5!-34%}Ea-=$juvw)dNSh4^i2C;5QOKgL94^PQ|+;&ME9^ONE0B!+1lMNoH5miKWR z=lYFw17PcIOsq9jcS>;(?Xw3RrWvFg^Hw zy;ttsInx9wnQXQ44=01&*nUIG+c#MbieTCz)2Ncm5h=@7ArT5O_%n z5R{=oUkCwFk3+p`bW%sS%KcviDJj~PMnpRX8C$1E4%+~BrjHd*3g5iSa91z6;R~f<|cLTI@a6h?#iz6_2my|wjN!ck1NxNqGTS1?PR24`_oW8Goflq$p+n= zHAH|%rg6JD$LXdR7^;WIy@~dkGaq|f+N=PYu!|Qj6V{a1)>M-7sqwy34}TOTATKZ0 z%dR@rQf83tH6L!h5yb2A#_xfPI&@8vr*`CsOkSZK@~QWFV~_H^_r%K4x`yyqFFt0GSbX+-o5-D>SOWAuSVDzUY!jYv-C06jYaiak*}O!*Huwji=5mmrF>BE;f-ygueA3!wqlLE>mi4ZS7#__>y__+ zyYK9B4)nx^H4EXVVw52TGRC~<65S3S`(oZVzq2AG^I6+@V=Vs>(snb^(ABQ+(&1T3 zXo=(C+tqVgoc7h+DI+PcRv@pz-6ptSz_T^-RkuXFY}#<}Z^~ElSd#ood5Xf9I3m=( zB)|%q?IryljJ&ICmAX83_r&ZsV^}8Jwi;KqSGd-SggS;#twHILLR)3$3rZiqbbvg? zc;ZA$D=XG~z(i5-ji&X@#FPo?vFbQEe98CBTcN9edUlmf)IsZHh04o&Sp+_fb|K_7y+;$b@?14}-Y(zD0S;D$8;1lx z%T&(@CMdFqIBXJJJ0+)rZ?|`Yo_-F93+b7CapW)X!@%b7R^x(V@~NzUiLxRF=LDeH z3G)VbrP#X<^nzdTYlT+O-Z2+m8KC9b@?Ac%raNU?8sxjedpetyLH$|-ujb>Gpw`BA z?3P;ZK2u6_`Hc2}oJrOgc8ct}yme8o;%=hal{3y+h+)xHVe&*6vozRvcZQ@1n-BQ#bGP zJm=+9$-+5yAAZEV;HuZ;Qp26L?bkN>{d zyo1ZGN##e5fTAlLE1#WrTVId`ttm`%uBy*nF9|vUOg>yE z(mv_U>H%C-6JcICeB@+?(Z|wo6|-^kcYI0_Z+|7p!UyU)Otlgg;ZtT;42^1Q2)XQM==Ux=$ zvxA$9VAl(2ra#PSM)2jR^Uh|@x%a|unaGZrl#;C&@B3G*ZWl^A!`};0g?z6p^Ci6l z8az;qiBq_>HaEo+_@fo4GWF==mc6Taq^rjHa|+q=&w(DxMJiiC>XXu8q*x$P{>-?LsSWV0)|Mt4SLbBUv{^ zd`^!xo@!<-q@6CBdUL@vM^!)&EOC};oFu16Q+i~9ai0dLoQm00DNPXhW7{G z?B&e`-`Up%w3O3HwWeKP(Wb%{!J)FD)JZw8%cegxjR&+59_WJnEP8gHrkAF0QN3~b z<#@8;Bbk~XX|y?BaFGXG15QGEZ~Tjp zIiIUC<^q@E!YIw4Wc~HGvJr6lw&y<-dzJ?pJ_Y!_Y0(wCEO!`_}5ed|< zzAA>6yzB!F){j*l90sAP4(%`J@LxlT)pvb9+U;}(cZA4$cX%M#32X4tPY70eo$uBy z(Ai2?R`_zH3{W%Q{8IFZh)%rQTyy%8MZV#Mu{Z@qW{C_I{T@N5T$^cc_ATq#iR0bP zPWSM+`s8WgL*9 zvwoMpPi|#&;b3X#g~|5~cQo>`;dNag#~HSzayj;1)dKx)&-wSh-vNm{16v9l(Y!kf z&hM6-d<*z~)Ro|V6DDc}Z&HFi7y!42|grXWYR|0PHyv{v=eM%P_6Yc8+p z?Y997s$BXXn!0u}?trdbR6*YFb7)c;{E?uN4ML(kEXbR2?b)CBf}#o$28eobtc;aE z!j_5iZcY-!?Y(?n4vf*aIF=Nq3Q`dBH;q**%ZdP6Oy8{Rch^*_iN&&mPwUKMBn(1M zGyQ6nM(4HPO=;?=N-d@;OU^hfZpA7-oYLNzK0@_QaGY{b=+cXsfVZ_sIwV8TA-Q$^ zH*Yb|=KNFtJ3k8JTYs|D85Y@p`8K@3%U5>}C|ezMKjTDvU1Qwn>~Ds|kMxqVeTV9Z&XD9SzbDx9+tR0_Qhcjtx|%PMDWgzpaSUY(xxZd>=PQOhUXJ?a&`uMm~(|Iz6wsY!=%Chx929MY= zATR8mkO;4uu``S8(pqtbQ^%%e1RoEdm+It=H`|5@&(C2~9pciso{&#gZuEbUhGbNu z`-0r%o(Gl@ukxvDcYku2r)xBZ>Yw+`e%bCq7@>7e&DnA|wR5J)pl1$gSV_g*P_6}E z0vtA(Jin98^e?W8_@Gk+rFK=r2c+y!)V*JI@0ZwonQ7#hN>&4W{2nY$rC@}1(8bUp z-{A%-H9G)Vzj=2V-|SkA3wsp?JLJW0~e>t>ooT^*paG;r3+(J?4oWPgk%@dhRvs+EU>O7(z> z?$hUg8iRF*U35LBn@w;Z^-V_#O47yoeF(erVH8LDvcHMaTC;uyKhW60=*+N4-GL;K zVN3jx3M-Q@9~Ey-+WDfzniaOq>}xp3KG)gp23}i~ud%cWkk&8fkXcP0fC=+N6UBj_ zG}_o?Y)|s4*Jk{7!ZEJ$t}4|kI9Yu7mifVj5T||DNTe$Hb{DXnX3&S=+TkI6FAr0JI3mt(Y`Kxh)< zW+~?j>I311`-N| z=X$VW2a^TP7!)OI2rQO??Y2s zCYN#hXGwIP9r@cg%H+O@&o)IqyUK+6bu;wvmAAwXbruNDM6hj@n+Q&{!QRfyw)<~x zSO#2FVEHCw4chVFs(9e=gkM9+R^n_ubJVY=N?7Kc_&c$DLaIZ*9hV_)MqwGJPFZvu zv>7dyw|_%?s73iHV<+jiU>vN<_T>y){oA|T6bE{96*1g2*ORK z)TO2C-m)GuIz1>7vXsetoZLVFJpIxJqhDM1r>{o2Bu0+oLu~LRJl7fms~Qdv?W*8W z4ptZDxP+~u`W^~IO>V*lcgag|s@2)3flV*fk@h!w@PiKR`8c2y0n^l19A7J-O*ezQaE`g|RIDkxCQDd+NYgSgt7#gFlGP z3({)ke|D_E7yXMw^YgBw<7G8kJD)Ri5e(+!a`Ok-r zlD{&@p-G~%qJIwdZwvJoi~09M=2Vy+@_O5z{G}ZHRb=Hqhhzr*mHdF~cqFR*mjV5k zQ|8}^O8;&I{f2B$N`|ba& zF>Lw^xh6~Nw(I`GIp-Z?cF@a=&G<9N_|N73%jf&|C%?8CsE$ZTdYbkR=WL|)tHXuV zlmC<~xDQrKRUOUk48KoJY}j-ohPxsPw)<1Fa$c2k#0jH9 zb}aN8OfTc`Ut`3gRxM}qWiU?A z6g6w#*4eK=iPWFIH}KWjNKasIO`{^6k)(k@ptie6S(4Dn_XQa0{UxE5KWhPy=F0_l zGAwYbd-zfultAYYTg?4ijjsQN^oI@wuC}N5AxTbAkPyKDOQo3)YUi5$!^e%)2tPQK z{4-XAhh27v=>ENp?fEiY0md*_h<9la`xQ=?w)D(M#p-c-W9(=ccW-x6l`|)o~ETU zmUBjVFaa!d=J2up$|!zljnb$VGIex*j{(NHGw&+%Y7v}i?q8*z?p7M?T;5Az1&B3F<{sOCGxOR8Sg3zN}mC!4(;tM11mUSGlvG@t>Tao z0rFb9op)`6E9t#hK@ms>D&US@Mx#geX0%QF$%Osi7V+!kmU56UEgjL4#=QC7L{FYF zhR&Ts`;gMhOEG>%xFU148Z0v43zJ8S=FkcIw@E@A0e9)kX#Z>yVl{v~u<~P6aBp(8 zLf0{+!%cxU6H-yLzYw*4g{e0g=ctm8)r!8q9I##nh5%ArC}bR(QAT!;Qur|$OuWa< z(1+r1_;x-D0lrTj2K(Gm+F8Jrr+TAw=u_NiGq^n^oIEQD4`p!p^{9=uarhX}bKpBy z(;ukJB`TG5Nbc)FH&40w~DozJ_dRuIzQIv~SW`U3AyUc2u1&rj;Jglw>Fn6sCyGu(V z&9Lsh8GluCCHbK$^)`odPAVDn9aS?(dmThcf*r6~Jx9{7uG|CxHK#xX@G_con z-H}-HUCh;XOa?kbY|^$9?5_PYS&NIS+un=pRTA{ZlD+pSNH#a+y|uTJr8y_k<>kcV z(y5JL2l(ck_u~K;PFH0Ca!>kfSA!&s_>hF6)M(Jb7}%8>%BVvjIS8SW4cpM&9}8IF z4yX>m@9%jh(89pGtk*%)JeWzXcCo_R?w^31 zZ@m_$1qMwiJNl&oGtr>BGN40lnqIw|WIl@u3TF~xNcogtn z#{MU?kCoBG$C;}q_SZAX1B-&?lv;}5P+SM^-eUPc4SCdo$l|`SF zyS12CXfNr*JT>#GtA^yppM;0#BR;B5C>LK#tm| zS>Z_Pfp$0olwLz$ftG|PCjybgu2CHg#;K+0WAMFmKuOoyp1saxrsyl&4NQkEE2&UM zprB@zb-cF&u&OPPRN$?Wp|TSl4LgU1Z}#RWcltX! zf*GUw-`ru^mB&;-s`L?U-c{`)-}00`a^FKqJM`WRxP4JBfw{BhTeNx(BpNv@K40X& zq{cDNS<1@i(sLOPM7ioH;r0$oMKV`0l{6D8I0OmU4Eq1DoGwV`@JrNv>TLgov{OxN zI((I)?<)o)xoanUN#f41{<|b+f}cQoNvHa5?q~vX>$!im3@*M+D_O2QXQWGmFMWS^ zel*N<0PTGi9>C3-M&6A8fKq3bC$$mXQMlbY9mk*P?#Q*DC%)4~wP_!(Wv(Toef?_Y zRrCT%)-hm44s)>-GRCLQg_PCs?`Q5-ZM2Kw9KaMCQhDJv;0b^o`=l+o5DP(7nC$LL zKz#bPxC(s?4GZFRuf236vu4Rn`URq z{Uarp>UGRXmRWt(MXW+2t6*%5=Kxw6N znt4PCd!%F%%PJTV8?Z6s)u%}jD;F!Q{?4dnBoQ*BP$tl|Am%Q@ms3XzYF37R7m#QM z4qdj1p#WZVn6bTOO{v|87%rbF3qFOrdjt;Q);>(wvSzelGNZ^j?&~@3=!%cposig> znY;f$U`)`{erIaY=OK{=vzooV*5w;o5Vzl+<2~^N1Y(r!zVc*M|6`>RHlT#7GEr08 zftBAwCP06di1B|wpUX-pQPJiG@3Ng+xA6h*YM=>w6EsZu>dbCw#k?uB0RJ2Ds`{D)#ijSaa8;{wlw1MmGXDk0ovF&5MTdn|rV<#(pkb3s#kD12ZDKeh z!bX7?fv);lWiv_Ji2|<+&4zQ;XvPRb`&J$(5T6>^$d!>g5AL63pf(gdIClzil(9m;rCQ}TWC+K}fuSZi2g&_dq|ddp zEdW(ovDdJ> zikti*0CDXsm??TB7V`cKXZT-~Zst#@CqTosffHU$`4YC5r?OI}A%pi_-J^T0RKXiS z<26h)fhh?$8jCl(m2g^hCRCLiXLw7Mm#&L@O_LD9kyX$?EI3Ex*X$6#)XV`!9uQSA z2~~?$O7k=Eat!VNhK^5h^Y8GN=nz4NdTd(H2l?2fGgwqbYh0%zsK_6k=FJ>cwY#&D z*#g5(W0ldH-LN z{57D#E2&T1**7m3Bne$_;+9;3thx5yve;K~NU7p(Xe9KLiQ# zHGUbu0yg&){vlZ7sQasfmTP;`KQ#LNF8`_u;i?xG`wu(FiTpxR&gP{2!x8=8Dg3`{ zd3FD1?m|)ji&_#2o92Wy@wVfYTToqm;iIYXKjk+He9YK;e9ntDt6#Q{n(31j>Oljs zZ!{nL&tta+)1KJbgSzt+vC$nBC#tRZM}pyBS86xAnWnOYb21Q|-1WBvynmn7qFZJm z{9cYsGt8l8XE9G7sqh`{EHR-Q^dBdp?)c#X3>v8K(yIqMd{+DYV)nuV>VU+5fKxZl z)Un#HJKx_maw}Kc#~Vj{*!fS0ET(RoMzJ34oc#_|U!Qm1{<^@Iy!b!obn&3r={mF! zcH{Z=9m?E;!-N0%vZC<17yHPn&T~$A`12-hPe}jyAOHPH5%O0&1Ia2oF6TdH`rmn; z9w)QI4|RarKhiHx1(M$W!h!$)1RMuCsZYF_Ud69?V$uzInFyK+Gc?>h%3f)?X!^aP zQGYQ#f?_s?YYbv0 zL-8apfFk=0!~B)c4tU0w)%f>pIvWlIQzG~17GHXcZ$ z*I?|^sRJ$Nc82z!qn$5bJpZ-sLI&jtfAOJLx zvFkleib8T~rfklYj=tp%_{x|CGn(Xt+(!@+4y(}p8r+e6>ZI2!m;NwXkm6*cyhN$k zA@*ABvlx>I^%nbbJaH+1h=V#skzW-qh>cgpe@%`WaCwcnMqe0f$Gn|aLHjBsNX!0x zoF|qS!2zf4l3UfV@S z-Y|}*6(A6t@Xme96>|i0;TgG1L~Xa*xm*q6npe@$Y%)r6_Rb-1=A@`I#=XXWASWo` zp@5|=0OFT6O0t7rAYK6~6q&dYMRrROIm#Qwq{rTs?zp1qND>o@5&qaOr9nWc%*E;A zLR5DHy_oj73<2TDy|;OUBwOffMQ?86E*Au?f33ja>1%?BDBor6L3uoN2eCG{pq=8A z?lq(UuWCBqBH-RaUeI8?E~c76yO#DiV7o)kX}#oe?|rg^cEBCzJ6EMWsvrbTAtQca zkfF6B^hh+^h@K|{#tM)>q}x$YSnWP*%O|1V{>{(-Y!Cp4A2Z&9?Gbb)az59UjH~H} zYC!vqy305^O`CcJZQ-&D2?F~dTp;UsK5R!O-H}?i%#Ehf$Z>#lJUfaeG~ipu5FGF# zz@o@alCX^N;Tdrbg)$O?pf$=^rCi)gUXwJXJXlQiT9icK_-Kbn&kOhZz3Sfz&gp;O ztz_Vw(_wPunB9(?(Gz=XzFgcaU|M?uZdG~JV{}T)q^f!~zl!20 z%em(5fI-0J@sqJ&7(rc__MBfsOk7|o&Fl~{ma&+ORw*9 z-q!JZ_w2y!VzG1A;r9;Q`~K?P7W=VdLi~N3=}ARc3ZxXMK_;UbLU1EM8E1k&0f(f| zA~xr03&ANLB@K!$KKueH-ENtlaHTgcTaGd*mUjPA!XDbg5LPZ*a@c0a@$Kc)pRTa< zTV=8VyH^AiJr;Tltf=1&4He5Fp9YqDVQ+`8YrbL!TMZg`Qd@oY-r7u;JS>bT$G!mI zBpFgK2cwH!&pZeK*u5cw>?w4>n zC0R>#B-d794TLo-vkRj?3|v-kajdn=@5$8IV4H$SS#M>_I$f>g_gd%;HCnwNM8DaV zEaAykau9U%)Wut`emijRFtgEf`*1VZRX6;F)Ar?p2M6;k6*k^s0LNO`re242cTZ(6 z6(D7+J&2#21V8y}det~pL~%aeG0rb7gaBCSdB?>8@2Poo zrHm0+MDyFZGnqB3)i!$Ayh2opzoDlwpX$oOvTQ%r)OXgr7_R|*Ck%p`=?$&5Of`kQ zG(WTIrHP7DGOK&VA9Jo0* zM=<$tB{(cydtlb@?>C?OFC?xo}52%q`!Ed3Pgw{1~T&NUYC zXH$XBrh4-{HmQ7ePrWQ-?rI^p;wiDvV1c;Hw#+L+Ck=aBXTA@3@JPLUMy66@qpYId z3UXW~>>!Ep$JQ;~R{NiVGhG~uTe!h~s_r$65QJq-IAhq$bsd3m82W5r(thl3yYfW* zAn|!3j84H=pC4VV8@CucO}9SQqXisMu94n|i$%qVNQ3>?X~U~^D$@z*9XqeZ7XLje zb&Q)M;^xqU1S97f(Qf2Krz3Jr7|CIKK|34#$i$QSZrClX*w925pOn&d$%6?cSk(kD zY6~sFi&_*D(Im#ag^`odw@W(!gwF^R1Dmd6v^0H8FiC(N^TOBC*6E;Kxls!~WA~SO zzmw3dBgZRWmK{{O7yFBldXJ>-5p;R#qu_o}K94%{k%1y2Qr@|e_{s9qIQ?Y^$vcus zOyV`G52`KvJZ;Q1W8<;dBtOIx)0OFS$<817cREY8ZS`CJGhK!sWmq%nu_xqtgzv{b_AFT+#FsSx6+F_gixmg`0j#R&pl2D>d9&Q+cz;`xR zUyccBsH0J^hYFplzsUSgdjDo0xOJ`5A_^q!^%x@KwueP?<()JVQ6g*|?}dz4TbCZm zI)dKa){YT{%r|!-OcO+O2$}^)#jSj|dRZ-MoV?#{nSKpoDTw3WK+FtD@aCgk$yWHS z2fZG8=^n&5Mz{f}N~`lVNA|jwbF5qEyf)SBVdIy3VN>zAf`>7PFq{=%N`UL|oqy5u z^II>=oaIe{CKvo_R>~S6#X8dbt1!wq=MPwfA6Jfc zeD}%0Zw3ecb>#oG7K-2ND9wZmjMzN&`OB^|M$X72|6VVHSH6)%V(oa1Xf3w@9q*Bf$ zSq}DMS176Xs@6Nydb7j&@N`robhWG7i{Lr0GHSwSEUz8)`=)$8bL3H$Q7H0@7W3=E z#KG|xx^?zL%8E(}@m=5N^`?ZhFg13?OJeA04Mg*N6hAiwzfW2A6$SO(DyzyG9>SW! zWtEmQ+f>YscT_L=-!x)i-~y6oxnTaw1P&`)U#@pQG5L7MdP~|FCI6T6%X`w|$mVdK z{?Z1$<3T)%U}Kgi@+bltG!=;%#RMPOCTdYu|2QWx#-c-^@2 z^W4c3Qdln>B>iJn=^3rcop|}_hv2*sfC+J|iYUK4hy{n1Mb?Y4h2oL)Mk7u_=b2<; zry4QFRLXB6gKmj%9${XzJhkkOP8ok8f=k@>p^ST%&|~qmSlrYeeDB8<(N7TPC|8$J z$cf)K>)W#f`r&%^7+Y@USH+?_NX-hDTJXsTHsG+luFveCe#tL}BGe}uOdhpJH(1SF zYDw%(wwkq@Fm&`?p=Ifg7nG8#U7xy@6Q56`b)B)^cwfMYr_d{eMHPwzAzZ957p=mW za~@5*c>`tslcafZO!S0TzbJH-Mo*7*{%zH`5EZPww*`m4I}}52Jjq|;Hs*@MOf!So zB4vjR#Hx;0K)>p)>`5h&E3^x_l3@`Ft4KjWI(7EEqQ`6{^=HS+`x}V@;Kja}+qv7r zQGZCC-`AO>Gtril);fja(pxyqo#t&P?YZ(@R%-BrP{R}u&A8`3xd~`S(fLjZYNC!R z*$*uC0%$K6#R^ezsVm9OsI*SAvd7Dewi*1_d)W)V6ZLcpgbBHwyx#%dOH|1QuQBOA zcf`pIf}YJ;PyfolR8VT_R+AXOMDNp#&sagjgB~hMDV_es8Ukl8_iSNP;&m-*um8U6 z_m5b2ed0Q*-3ujABrs+u?*`csm}#H$4n*2|bB4D{rex!z-h~tu35Jdm zzs}BXeaK$YkuP?aT$$W1x0JvN1?XN@A+Ls%(ov465j`o(wVqF~K4L6BRaF)Rr2#GQ z{||fb9o1yE{SRMt)KSN-pd#Rm3Mx&cN_QMVItoe$B|zwr-U1oMQKTzKuL=a|(raK; zAO;8#LJK89LJJT==nz8Q6P&phobUBr>s{|!zq01P2XgW}=h{>eO9WO-e*gE8z7r-4Z%mRNs&|Ae>H)j9nSIic*DQcE3A^Qmq2Pcnc|0&Swr z)CnZ!c(hJvmi#hW>pfqBAFw2AlBMbyR_mjM>QV7ih4ptly2e>4t1uHE!YzVAef1Pn zRYO`))+vkIGxxJ<;derUp2XG14iZ9)E||a3rSQTecePr1@4J#*9WlnC*vQw*OfrnV zAg0|ljUQ+ZEWvf`morDC z$`+>jMLKeo`BCyk*;qru(7T%!{*mW2kG~!j1}=zo21!!u_-joeEJ`^`D;csfdlVxN zjnSg@cha%ZWtN`?ipsyJgVo829+;9wigY&NLqj5Vu6NV_a6EU_uEFkwmY=bMq2tII zdXq8ig1NHe*^W&s$1TFW@p=64)Gi>_aEcEBLREx*XYD}p`B@11Y%hHIi%@gUjVa1yJ7u_)& z{B6waJZi0UAj4+ujL_*$$3XXO_9|UTJqV+3W-R09q&rR=C`T?88cQNJ28yFz+sad~ zA1au7ki0nwLD&vmiZG35c8#qSI_IxCu9e=V4PJM(#_PLxPuo1%xE72p4yv0z6j`32k^#;&P!D!4Y<_5c4p09>T0?Y^L#TM9|$_-_;1{C3&SW9ela-CI12|IGNW+rPQHBdGlf)gbHE zrr!UTu}b;BHvA*1e?-OBk$<%6f5`?^x_42Tu|PHUweEH~`XJ!|MDwDY{UsN}&!9Ce zN-eINY_oVgC7&-(e+SOzY?a)@Z6ngvTuxl&b8T2pyMkOzm>Y>NRN5#zo&NVrbU}X3 zv2=kY%3zUi$-%Z1+qyS$-eUsK5spnZb;z$S)5_J3&+_2vRpQ3%o1>t0J6C(GL;rrs zT~nT;7V0rw;I{Nl%Xi}V^HV~6=}G^^%5NILfSjYih$a-c{q)bR{*l!`%CNat{@FOd zoB99OGE5{a$GcJcBnqmh*StPAj+2W8MA~wAdl3(k=F37Xq91;MG_}Ty`oH%i2l2C# z9l22LNayvUtXrfyfu=+l>L{m)n}C&UM0z%X=BH?IAag49~kj( zoQQ`eah+Ew;IPlH^1x=BuI1-iEOvE=s_^<=U0bAbaV9aSw|Di;06NL)xY;p*f?j?e z@99?tkj1j$j*!Ca>uvkD`-7LheF)4SabiQMpiN&9UL3`QN^gwR*9c!TkxMka4nNp~n^Cd&#e!L;nsurx2Gcx^>a| z)Xv5FhWal(!$C2h(j(CetoL*CvIyQl#L^$r6h4S#1T@uGX3xR1NrU=s&9X2RA6EMU z>&C7n6#N)X!B`Vb8Lv-6PXoP(YCDkm-#I*z#_c(m4{t7s z{#EmATj{=+_fOCrcAFg!oO9Tj8GTE`cwg_(qm+Y6MyoHOQ~!cuQ;wHF+Z;?Uo^&bo zD*0LF^}&SCekmeXv{P31kC-QlCKO)&>aw(;e2E7(7I})fwi{}-uWje(iU2J5NeD+n zP!?R6gHLy~Et%mzUEab&|6|v3W4nr>+;pu#fbTh5d_oB^9<$B=X}uT^PyT|HZQk*%{?Vw%E4e4{^?KofIIB& ze=uTbvh_FWmoE-44pcd`)Uu}T4KS1?mDX=;-uTgkuv(`q~cSkbD~Sd zRU!QX3fW7Nde`YP&Sf;17hPJZdEMA6CjI{Usgy>?96xocTzS4UzA=Q;8q+HNDt@Eh z3o~p`?ByJDLNgaN*4OeH!3uE3-jiwq5LeS6VqH@s{@P_eEgvysUKE{AbYn8H#IGRv zk(Geau+CP+I`W2Z2bk9)()Gu`VltrAf6SfyI2zpkG;8(=`B1L@LWM8u7=`sAFydLz zaen19y&N(0B>age!?#lei-#YPKaO2Bi>~GIiw$1XR)fLmwZ~9zE-dKWAQf-FY@td*hIZiOj z;+p_Fop^-y+n(fQUMJK(_A-Nc3NBs3Z(VRb9>befxTBzjJ^Z2XJIZ zi0zW!oV*Kgcd<|IT--XbMhlUofAzskgoP$g!yHL|ESg%8UAmp zT1v4g@{FV&6HS!<^zr#ItfNzGFrj=nSfFXzutWeU-}UMF(t~QOGsL1lN;2#r*P(Du z)`m}GaTfKgYo`qLb+X(%mon7~j)Xddj`h&w=F3Jvqfq?1rr%Q#P>Px3M(-I%@W?~j zrpL%)^s%|g$@T)OIwAn)i??Hi*$9_ zFlBJvT=#!InAwon*mqgfNJH71PlpmPDQwQmmFpmP!QP&BLq47KR)OC4A?ixOa;X?e z_;Y+A)QAfY*Yuvt8zHF@=U2nEQ{;35)DXCFl6q#Bfm+%XQKJN_vYCs-)+{KK8Xp09 z#gVc*Xq_j13>x3v5CzR-QEfxdq({pshvDGp@wvSGmtG|X44H$cq}^1D!faQcKNLWN zj88(pRh1)GdHeI;&n<}}6+O%Ex9*rr-w!zV()Q=_91rkk?r9ha!Xj+%)Y`t;YEROv zz!~C~{wJ(CWmMC*(aooD{rcpgrw?;~2mz<21A_nVJD~zM{6g{a9bV&1OQcf$TCrE&r5!s z+D7t-d@5~a1XI$%I)tqcR4XicnH>hwbJb9Znae=_`u~Icv%JJoMTbx?|6CV1Pk#th z0cva}#8v}4rygC3q+GTdtK9a(QhDIG#ys_6iroCkkM)ueadX|TE^Sz4(bvlET-46& zf4jF9%@tg_aUI{y)tw&zH4W=7oOeJy{BR~t@`UZ6YfApSy_Yx+cq53m(#5Cl_CtpH z>5Efx#<0p-`%4#zg56DRjgpu6|9z_bPZvsM;Ap$At}S!;o5w>i0E;;?ivs>;$G03^ z0d{Q2Bqqg;XRB8I?GQNu&}ktx$gXdi^e$kmjfCVWa^JMpm#%G0mF$6KcTWga?g`2AAY!Ea7oFGQ)QR?8BkOq?g)sWdflg$MPR`AYb&GA0hJ1u$#%=Bb#VY#*3 zTJPibLKvxXXaJ_tQ(&6zyFBB(Q5D+|EU*-$!sC0X+rTfsa$(rsF1+L`x2PepY1=Hm zANlcl%gOY_P@Rajd2P}HL!YpL44u-XIVHGJB~c-vWWitA?mw{VL^~dhOY;y6#FR`$XxMVc)aZ;Wy*n-fh^aTY~3SD zDLbo7M2#wL2pl%Oeu&r8P113O;X3{cNr72{)kSI*5FG!E?NA85(Iz@WJ( z#)9w(?+e>ESv-gxm9}kxfbQ={5>8PEeH$z1=B6NHd4|sKZjD<~uYkzzLjQjC@wQ~h zyWe($fbW-BOVr!EPblMZ83b|t=NH@kTcEAp4u+Lsqb9WLp^2&BnnWIO&8h; z7q#R@pC$PIUbo&HVt7F6bcwedV*oE|Gc+_^_ga_eEnDqDO<*e8{E-$X<7k(`$Y{we ztWS`aTyM{UkqnCBb5-{qJ~J?T1ZRBtkJ9L04;~sP{3|0`+q?YO+n^8ek^vpTKc(dV zVLz;sqM24&{LEHh2iMuQk{!1NQs*4Fhw=!n7E7)1`MDXtg>%5}>p(|(PtfMiibmyl z&9&;EWA%l$vyZ)Pmuc>0Wy@Y-6%7+zvF4n9jHL*CoAi%k*koRL-vuoY_;R|AwFXzY zk|G8@2-*S-|viZpJ{H!lc3?Og)L`$7-cy$8f3rmW3Gs ze(LX-4HbNn-g)omcE`oQdOBaoCEmSU9|~RdXbiVB(m>X_q>NUblnsW8#?Q6HZy(X+ zk>OR+=To`_uf^F5%^8s611%i6<-ny8Bq4UB!{@8FN+Wm2fLNX?(ZkkK+zI@zWp8UY zn(UpY_o*~V^q6sE@nLZGkpgP5f`-RDS#&8;tL6-c3b(>g=<;8^0Q^8UKbWphIvc_v z6x>o!nenJ{yB+|-$T0Sh!oQLfwi;BI6CW>%uC0gQn#>r}I;Dn)nq`p}8;bCiXK~Wb z7T2&d?-Tsc+LdyACZ6s;a7hXh;q!|LE&|)FpHi*u58KttY-iO4Wpzi5q6TQfC86^C z&T6X4HdD&2pJt$OnK)euc^|(OcS?JuH2!pV%GQ5v``+e*)B1q7pPR3ee0b}t{Otq? z1EaWJT-vs)zW%~LxB4QUe`Ey|_#b8XM;ZQYZT?Y){|l6%T9|hH%zw#*IC@L9Ol>=8 zlw>)9V>y4Q6Ta2491(mA@)J}d$~OrybFpYR*l+j+nkQJ|zPW%fY9%2_`;PmzYSz3iLx~s_|yu%3XB2Oi^$ATLm{S5{>}WG~o5Ktqv)-$%860(dy>e z&#F7!*nltNGo7CYQwXKz5_YhFY#y8PByGFEjl015p}sR3F__C7(Q&Mfn0a_YJ|_of1Ju> zULj)M`k@5{Tt&oJuGbdd@{2?1TE(YKL(x*Cr$42WUq<0VxnO5_Yi4RyTD@T&t*MF? z7&FzsfPaTMgD0a0t_qO~vd?=!QIn0dguy_oX6#OPO-;l{=Q;%|BPU^qjHJfpb5rwprA4Xs}rn;;<` zn8no;?2ASJK2;F@@WazaHX{^gn7y6A zCQ}O(<-_3}|JwTXH~eh%7b>-+4?n58a0~ApCbYW$>L?&=)Rmo1{SPDp-#u4|J++CB zz;}1X#Qy3fCW2X?qF2TLEhV!hQR6n<_XT}&{SJ^C&x%=V*gJ841&k}n|A{k2O-4(= zejQS}|T=bKi2AJo$|p}HtFjmLKuEXmaiTXwvBF_~ud z#}*%-Kcr8jA8rs*tSlN#D32bG)s~g*Hb;C={xCj9GY8aggf?o<4q$;phMHtC?n{nV z3EimnJ?Z43$KkgXZmKkrwG}uhD@kiJFWq!eji>U_M;rY`q9+YCA*Dj-aY=~c(Lf#- zOTZyj+?ekqV`vixAoSr|i?Zjv?}Ho4SE*fwAR*}|>_f{v;A5Y`EJ|jii_B7#?9rBMIgS2>S^W)(E&NXlk;~r3VRz`( zb*!dR!IUm>1w5{*{7ky-U}lpFmZ{lA+!!o4CYNg)TAm32HWIFVLEbzkTQtE=$aL;m z1pFz7`1!fnJS5#v01uD>&>a6Bh!^ei>O7Depb(&QqJDi+{RQ8R1L{3arWWid>ej)H zA~Dpf$;IPNVMNrLWli16@PQ{ci_z3_f`S&SqsIa>+(=(wr361Yz}elOqColcY@ly2 zXpQG1ncp*6se#4S#ISMvxvM@qip1{A=PH*iFBl_dd)>-fg+uP(=z8(!h^#k&vL`G{cDhd zrrAhQqkWqap|{5%ZJFc#)I`y7)OC#SYHAUa-(!sNeRN4MgF|K+Htl{vV zYxw49ZsTcl1%9N4x2%~Lop6d#uDlz`oDoMwwEuQfGIE0K2sRtrgJNs>sQ?= z2b$l(N5sU?sj7bq-OI#W4c9`|T*`9MvNoIm&w(pN)kxRa`wK|FcEeX7^L zCi%5`tg!Q#VOEnsA?id)#Py3y0}$H8K$SY&hlXqTZTErU%Bb+R$)rbEp`uX^WB#c{ zR(sjlZh;4x$22NV*EnjV$ce71ydz&*UbqQwZAyg7P_d;#@}MaY37t9>8gE}%eZA_v zr3D*TS#0tKFRjMb%J@U6<>H!!;Rdf%BBs?3YV+xn-KpIY3U$QUfy`7ywQrWC^a248k6^XX6o6O zJ3)%Ox#w`fs&6+)LCELY!mW2wPtTkEj0+=2n&tm7n=U9VXt!$@^Nv88w{jgNOJge8KntXIU&4-I3;Vrsv8C#@ zTO+St)WzbnrL0?%Y(x2Wj;@10^@)7G`&e+I$+C4$%}sGOHM2<> z+@^nRb4Oeu<}odE>%04wYrRV9vY=%5{^s~5;eA$EF?`oJWO#uaY zGUATEeBnUL0tFw1c*OiPkA4!1qZZ|B9yrMS>{zc6iDz0kM;fel=xcop&TTI>!TqMY z<~7Z!t<}^wT{%ueGzZJhopQ9t{pPE3I+d~-V=n`3^QC7glDKPTid8~S`}rw*juEn~ zBAOUWyvctiUg<`aYN{>Td*EeA1J|1hd2Wz1GcIIMXrfTi05jtR{hd z9>v>abHjoH@tOK{v8VrP#{&UVpT)_03own;S3-!VF;i8gQZj8Tf=Pwwx!ui7`3por zCt^49iT;xBn{(}R>0Tw>ZwBaw#y98bz7OleuPNZ?g?dUmSKg#EYfO!OJxWE53ZU6V zcT~7i^SyDgc6OnX$&5=<3X?XF(#~agRSLHwoTIIxk5}Ltc{$$iaB!@d&yQ7sy%|<> zywVRc9av=T4LE$>c)YO;qj)z{KNN{q>^9{PGC5&7c|lemkZ=0s07yHMf;>7{ov%~k zyKHIfv#955-q%+;bq1`&-HFtBM zuOh>*DYtN^QrxU@atyb#@S%s!{X&W%?;ZzQ>m~?0JyM(tmsG&h@!d!u;E%jBunHdX@c0&L2r?PwsTBL$Xtz7|ArB67qxxOC30$NrxCGa+tNB z61x3MDWAgJX`?9v1N$)Ot8nnm1LMZjh?B9W2AvYAB?5c-m#%cLw)<<+@}Qr;|_pKtIAw z{nZ;HMCD96?^CGCnN*6BT^L16b~+)fab8$XU3UpYijpb20 zHx~lnerT72jnj13^vgW)3Uih9n(VR^Zd-Uaw+j21#X^x@CARa2Ka`$F;f%?K6F<_e zZ@gYa@MlVtjSyZmK_}Z2M2+m=Bd|=p>d{O+T)4Qhv6zCcGu@i(y!XYCHnZ2ycrQldXHy z8uWt5`dlfJbY_06tz|OD7>!yUjOy|Oc)B%%T0PH0c!``J$7#;kuNAUSHt9z!=ORj+ z&?b9O-^|cQq$6QZSxGdQyz97@^QRqTi}7TSkO1c1rrrogPMSt?(x7W;pN26x{!lWY z7xLrfVBG9vRuJxeLVM>O^hDri_AAPy89~+%2qr{Q!gkGn2A>6+u0j(asi0xcu*xQSkQ31bPo9Vt&C=4@Z#|e|npMMzVk+JM; zFNfzj2J7a{`l2A&PWBq=VG#+8#O8X@u5kKYdD)Irj@T6qtH#7Ka_-wD_8JUh-+LK% zqhp$~yj%@$$+m4^p=bI(J;X$6g(2UVdL}Qj6K9dYExJ#OmLe@cFwBPVXZ(?jrwSm; zwpcEaG`7@4Gw0$z%g8*1Sj39@F6FIkIFeeHU1b-KS(PW@WUM8FInC*5 zP4brov2Dq??d!$hT;crXgy=O(?*ycN47GfUpWHy!!p&f0{MJex)CN0qQ0&;@Xdrjg zX;TDQV)5zSvWWIkieVTC8RQ%vuFbXJ(0CAHj0M`Xz8J1(D_fFF`LDIGR&cgXOZ; z=C9T3Z!eQ&{f+0BSbo<|)Xo z6$O}pHr<+fs*OyFQ=@|+@mIaSe%%5(;=9^`NjEXSgs7Z?}F{ zXjx1=V&%;l^u*i(`zv+hvTGf-fBf;Pto}~or&HG)MjdYkyWb_k?II$#@;CLOTFWrJ6q8h)N?A2jdH*S=Cw=f9s|8++=x$vZLk zzE0(?do`3|AIY6+`_FW@zd+mE3_;c!U9mS%@sZF5DwuumW<<<7U{MZCh`oKt-DE?l zk5!uo>5_;6$uiVKZfC*&!zJ1&21Od&*FJ08hf7xI&O0oS?u=p2DbOL-r z6!dUt9E6k_+>HKq@SbJ@f_InEv)hwNTj2_DTcsMnFL0DH&3yIM2iviLm+TU`f+X_x z$bb8oBQ{ciI!8t3iog8ogTJP~2?+$a0FqiuwClIg0;)2g4h*_QKivkAeSKe@0>Fx^ zatZ$?XEALMurbOL3p%zlH@8g+AQ#OiCyZtSsFcbuseucJ6eeaXX&d#8w3&^2-3ubA z9v*qi?FYG%LFK*)FDO4Pq+{e2xoc?Uo(t3MVjJ{7{G6v8dm4Xxa)66kPvJ9e*kEuU z%LVE2jrmk`B-1HWHHzPw!TlE;p&si*AJL6ytPn=MZKvDbaCOhqD*>@2K+L#^yBCynKUpS2)?!-r6Q)qQlW<@`Bs)fz|v2j^lJHtCt&5V z^xWkCD1l6Z(mj4<#uAlNN@L|l%v*KYr6-)Jg%lD$Yn}!o6c6mYcu^HyIUUmcQdRYu zYpV2^D}Mu7TaJTP&oYBgMlqUdy}fY)s5?T8(E9Z*X0_{c zA@rfgtWZGYzPPqH#XF85=h-+Jqm=oyyIB)^ME%AVyqJqU#hOnVCf`N}qS?3*%Xci> zi;V4o0N4Mv3A9lgvGKwBpqBKKdeM}y?=PTY*W&a0-3QIvT(@G`dZkLP^aQpsXn(A~ zMf2Sr1mtLiHVt2-e@s8zco+d!H}UNfTC84;Ky3J-#>s`Xm-$fgqAA`p6zSCrvEmD) z>*Idc`^VSpVG-JJI9fTV{pMi>ISyf?6X?`+LPE7ipL2x+6Q9~AqP}vnT(fd4rq^OV zKgx0OcsjNVy!>*XYieX)CAODJrjG3I){V@S#pg;hmT9~YWmrsEQI#VlI>XpORI;FQ z(wI;;SnU}>BKhgBug7{?;|?eu8en}gD7A5hueL@Nzve2eIqyMqUKSMXOM2It9R+#s zu8)?P%HvAybqf61eOtM~Ck24cV>o|pOr%2I+FjfN@9hok$x#aH2`CykDr(=IH%<99K%h}pH|vC?mA*qbvsY3AcR*lM3hItz8f!N9HcLAe zv{DOJuWf}o1IpR?1n!bxseX&n z?5O`(p8z$^H_sX-iurEocDjUL?mmt7r@xF7gLS6F$M zP_Hw?6vAAT47IQK^u9lU@~!ZKwfRca2qycKyJF{&_rW>Wsh}Ah-O(iL#I(Bh#HoHH zr@fJ$S(s!{n1C!d^(1$_*hm$AyyuWsfxqbq(pgs6@SwrJ*>%cdADX^?H~;w z*sw{-_k8!;sxxNq@?9bHx!i_rB(#wWI$F~PZK90{0z`+iA;D51VH~6X+4bL-mMP80 zSC{)Q?ti0*(zZ$Xv_42f48D;FMs*->q?hsK@_0kIzcL z$Yd+YvJsP4_Cib1*r57iP{&0KqA>-+aDD&F z(w!|I4x|86VJV(5n_26e!Iy?mdSLex}Rd_-p{4F1|0}}fC5M1M#+`^+E9STo6(j#``A;7p98G&=&a_zrw?ngDCou7hY+D?~PH$azUFIit)c3^@w#yqJ5_N2V< zWnv!=S{`J@)VKsQCY=dd4Hzv4W_;sDBH3M`%4W z^&p;Sn_fXtpV=LOw<0dsCTy^xc!~FVg;e>7(Hix>?r?Y_mHhT*L|ZcymvU0y4CNL~ zU@l#Mn;`=@KDRWJDQbgDv3*U94SG+nV-IQ=oPW4B_q9-p=;z0bG|39;(1;xYE5$O1 zIps&m96z6|;A!F_titWAQOlU=bny9Hs#(FBi0O5}3k|Qk)<>H>>AQ~9N8yDN`ZL?Fi?9?MR<~!ZVWu>FT*x{l#UL}5Ej`{z|TIy)-Oy2Xu zStCW`-l6HIKNw%PZjI|sD}^Y6+^h;aHFx26w`zhJ%BMk=R;vqRS!Jga11>7q8X#oc zoml?6h6imI`(NDGcQF1S)8V)GkU=&^rrMo)Mjo^Zi5G)r)N~mKpYH6(9N%(c=BTBa zA46>Rws1WLlN__Xa>%~6rz6s$sJpoHPTtH|Ggr5{AW~!MQhEMEZc5!SR`1;wUqnfL zC!5zVn zqRLc`;r3yCyF1QFg#Cy&D9*f9ONe|<-9CXM1~FbGk2F4@wRY_}WZLoe?dN#~=9)4t zMIKif#?N>K|hJ-!`$l!Ijv9%HJ2V8PpRACAMwT z?!W0oDZBV;sYADKdm?-L1AkQ_fGErgfBpI2^k_@^8@T=FMElHq)MU4nmu?xcsWF#~ z)9f#+EKz!-e3qt9Uu#?`q5+StdCs!UQoT(P#XCe5)#zzwq~Lq zENMBvU8REeWr$i{uCO=?<2bIj<-7%A)!g?ID3iN`QMUyM-vhSw*W1Dy`Evn2nE`HP zzq+<9|6jZCe_R8*_gME{hs(U-?@A8RIW_t!Dep5gcOxyxGJvx;b|=B)ec zE8#V-h5E`{54vowlH2Wo0`9rIWO4S|>D4No`N-|uJ8scE9q&9_>1X6X`(xuVX!gt3 z@JE0^*EB#s8T5R0k=J|QD7=!0!^x!JFpS;97Xae)KPyKn6@G_HPURX*0t{Ru0215vNQXgcRB(*e%e(2>-#&+)S@7nIbZ(F zKj}4>&Ht0;G2{&kb@jEdE`h>I1@&O9_&E?Qr6@LnG=$YYbV-|>B1lf*@1ATa^~)8i z2tqBb=uMZ6XT?$~t7TzTjlI2o6)1}*Cw&^yBjnE>R#?CZN)NzG@`nMie%G_-^%U^6 z#rr+ptEPq>*IG4cXF6=%J4(JBD~m9H9@cWKqIlwo{LXn}PVgcp2Q z4lB&gvR!*LyA$SuM#1pFxbEJDa$|&?cNl>ldtsJ=XEA>~G=MOfF2>IonzCCFB)N7= z4Znx^3;CMan6Dk|9XIphE4aNAwK^XKyMkCt>-9*~ZaA*JE?eF^48cSrwXJtp`FwK4 zDRPbTk=|;?K0Yk>)1m7Vv!~>JY~PC|yHI{7YlU>_8w+&&PUGEJs99n7Auvec>V?SM z;&bBiGo89^p^E&DjKRVB;6;Vv8*uCT$B@o9U68qv3UY#dJr|pyQ=Pai$f4eS(D|_o zbeGNScwfvW*68{o_>}qdMHN6b%rNLes)}*Ov3sr!yi>VY;Yhd?vYR!FCN+RcZO2!W zBf%dZI$InI%$IZU0*nFHaG)nK=E4qhY&mC0yWJrti;Jl0xuIQ6jrkR%I7iK_RZU0} zGA72NnL%UuvV+=*{tDRoGD7_68{d$?OrtQCyI6)}UcL055*&vd2#rxBqvoA!&Md345Pm{Y4coYIl$YC<2RHFl9j)C-V)!ihg!51Z#|YA3Y$f5B8Gj{sY~OYa`rVNsge zz30%W6JYJNepq{8H?VM}S>A&rn<(i4I7iMO->F;(nDuuadgY65;tIo<&1S}$=Pkb{ zp@t&`@2r3@uUkMn)u9+Oruti%=Yra?G+PU|e2E^c84Pq3Gz8^RG>vCRRnJvEc=y|d zqXda=#>5_a5Y;0%zQmgE#zcp2Lx#RD64UN>eMpQvJnO6iouYJ>+kT4UaUFpk4sGXx zMRp~pXjivtBDvHGk7yoGo618uo%LQ3RsEN*`uYUA!kU%E$XMn=tv==D*MI4OE4ZLJ z)&J;2yP@$611(?F_ol03u-FRdi$%k;I|RRbB6SmU6CIiAW8!Wfo1Lj$Qs|?p@|Td> zQe95(TF!4dbE)bNKm+CQHq|*{(D?Jp!A&-Z zs__Vk#V%7-7%h@0W5sS~s2a+%bYnmo5RD8qjC$)knrcQ3o6~P`W+Iw~mlR%cz48KmB;)FV?lP4Bh-t}KF7uSM? zMe(HvUcmp5rGBOmDe915=`@pMf8|d+*oEV2v{PGJ4jW4kC_mc!kda&xj8hm_r)M)g zFf-ZlTCZf`(f3L+9#qpoU0&|;@Zt5Ty2>v-h($hG8Lln!#$CL{DP(j4G&%0My=z*0 zh(`hbi^)25ei#ZGH|Nf<2sMYLuQ$&*4{7%ujeO1%dF0%$;t)|371w{vqW!V!w*ha>RH~`RGU%y2 zNbs~K`i(Ob2sO(o$(||dGwCimkt-UpyU=GNG=TgVS<6YDU&>@L=Tf`qdQQ8FPa8O>zNB zPAJ?j5_Tp_|J6+0a`=mK5$+RNQ#}_RufS5hd?Y`vlT&BHO_zIxS3Twt#W&_Yn=4+C z?cP?9nPd8(bz?6y>$z5bB2i@_9w8qc`a(g=dYs@^l;ttq&Tp;SUcp~`5>i*~JJMuG ziG75pk2<51BEw#F6!oPrrjty9)~52>J{*?K89F6I;>w~XT(ffVkg)WLXmc}ERGmhW zE9uy}UgOrdxhI?{AtKZ|&6_nHmSDZOPcy<-U9Kr4C1NvPhC^M|*BVI+a@sPZ#sp3$ z{D_T=>|LKH(1a736}&FJbr^Y_`WM{Ajn=6+?gaTPn&(_Lx*^$<$tPSjUF+~ef`0&D z`4xmF!dBd;>7#4(N7eItxE#)kLEA$6X)9Ke;r^2p{$lzJV5b~i9%KbdPwgdPCX=p> zq8xr;jNl_qN`$u^n)~7^H(weGZog)c6U_Fv`m~1@rOea|3QWY>6z1Q1iN^acb*BzK z|Dj;UcJ8UW{A5C{@Zv2W_fZwhy<`gWZbx=N$9Fa|gVYk7Jl-ODDo_3j*KFqF{$1!! zbBVsuDES^U`@yG>jkyX|tft6I0qLcN^FJ|{=>EX%xo9D!QDC82I4j3wkiyX^Uea1} zt8dtPPYwp%(--#lxlwuw5i_&5Qb4_4QgnWd3Dbx21d@(mt@-J%S^lf;kABO5IRsKt9`^B^=4-I~l2(&1PhImOJT(mKc<;$q zmz)gsDJVp3*um|hyDS)>5Ai}U*3A$!hlY=jloT&6gj7(vk|=uhRaVoLEf(ujg_a_` zJ|n#JuA582h1_e{UWX_l$j~1*oF{%%v8)KWzPLahUQ0EJ9!MAqxLw}~2gVoLkKctC z5vwQ_oJqo03vT-DY4YEhi1zX1OtPbvfZ_v5@IGh4LJ6QlglQvus4Lxf_!}b%tAu=0 zd(3%LKHnX*ien&RLDhT>b^l&UdEJNsqj0S2|dc*$%P6VG8qA(u(n~CG&GL zFQQ1A)fxEH`K2Ar1eUzd!foW|eP$Z@7BSHCfgsYxX5qdz#wAu|DuDNXJ-1x$fYmI-|RI`89|Jvw_ zZ5~V6xkvpt#F=_PjIdCSoK`P1(})UJtRHQ=kX`lOLc37kMX}ZGG|R}ZSj3by`*<0j zHLcdn<1SykC`E*y_tiQcNa>1Xt&E!+%PXr3+|C+dc2O&=^`C{CA<_lo<6~HJJr&j! z+NwSH%rZ#xf_hcHDN-{rfjg29;fn7=Rs>?g9^#F=GstK&KMpOiB;N?Y2eBge0ESXi zW^8J#i6*15a%L^J7)y!H0_~3jsUFQu*nY?P#)*VbZBnYdfTvdoE?<6}L*UMrFj4Ku zqeh#3Hiy_9Mgqc5-p&%2nXegdDy8T`a3E8UTt9tsJjwkIQ(iCfGBg!F+0I5SqkbP! zBltl(c9>S5bsuqVKVKER0DegCI7RPvSZqYp<*=Vl9~Y^AA515{RBabKePk>~+_nlw zgbaTk;VMF>E`+UgwAG-G2Kot9AzC<#Z-AD{;ztvNS*x^pf~$=5AhgwqAQgiD#VX+s zKZUBkg}s=21=v_}gscQZl4gpLiY{c*qebb-`{=92E)uqN7rEbNEjz0oFXl9Laasr7Zx8bRYJnS8Kh|Yy%&nL6hbSi265l|S zs&~IFn0!G7r(V|pP==}=9!8^xzq|T7>n#0yOEZJ~|IF^@NiN?TkHKMoW=8?a*3<)~|~*wOwO3)vxMXvM0#vsf6{NTe@BF zKr^=A-(R=muoSxI(5m(H#whBHKHk#Ld3?YUD)J?iIn7SbKg+(tmWd5-K$YuwLspN8 zY+|2>&NZ9l*|vVdui~1&An`=x@)2e;aqlla^)LIK>g1au5EAzdt-3&UxK>ai5og(T z?>?-f5}JzOlO(z2j?^WQ!t~9&MZ1kX-99X-(lmqbq)UbZl;GTblZTcbhZEc&YgB^U zDv2M@nP1m;=Eb3Ag!t0%ba0#7l~=~Fb@z1!0csJ})uV<)Q8}}cJRCeU<^x%bdPOUe z1JP=HFw(cxAr9W}Z6tY6ubT8BLN2bWsBtV6hm?^a*C>X%_neNtcPI9O!n{y~{Kg}P zY7B>Sb45MqWU?%V(W~AS31byR4w6T+Feh|4wkbr7l1!5-ti*-VNFJu6c`?W0Dc3b` zBg3#l?wDw>)x6witd`*D`=rIE6q(5iSmlHf~3Anp&i zFT+Yi8fS|7^)0Ynirk=G+*!Xh!6?&*34CJ4vdS$IL$USM5^XEQ!zHmS_(Xt-^9R?2 z-!-B8?67gyu1RAP(eGB0x?{W`&G3AIcNOO9gX9O4nb;P1l( zZPZ8nvnVKSycLbsvwBju{q7ekKSiR^F`d$oD&rJet-E_Q(I>M@7y zbC069RFFKfR?fxGOz30Mmdd1>prrfEZs$N1nM@~{A@!(d>@MS~mqvyFie6IBZPECl-V(@CXBWQ_80u1T2QBnehccyg1DS&`8rO= zxhzU5#D1uAeoz~)Avz#%+CIzt0*{QFfkxb{hlxkyZURcHP!-v&NkjM5BUrr;7L-9b z%|f2x)*glSh`6Xaz}|Em|kLGHfc&cfY? z?0a7l3r6xU6vejbbaq4%rRK2}Boj;D%aGETNu2)u=D2I-rncTa1%noiqbf}+J?6H~ z6>j?no#d+CH-9jI<27W zhM*cc_GhE^&27bo-7g+EA;wN|#ebFJD&5%_DmDK(PE9H`blW+#4G?E5qaB+vWnJA7 z^tVs8I;qdKL94WXZuO6>*qZl`GW?&vangM24o2Uv`#cc_p`n&SPR8(_#;r55D_3Kt zAj7i91@hzjZf?u#-Q#;uYiPxdtopa@0T-9le0GpEUwjGSzu1?6w(~kC^+Ur6w4pfgi&pUYxSL!)*8Pt4y>GWu zTE6@KC-@LrDDKYcS z^{0QHz?*aEx~y|DsJ*GQ-vMc`}6M+iZtu#t*& zb^f^#1~shh8eX!jl72N@Q6fmpiIex{NY@)s5Dam(vn>SZDmw49lVV4trn*&AqENP; zay%|zZgH1-%;@n;FindRc#5E5jCLAaBB*7~5Gg?yOyzIs5X$wdH3Wj$VD4==$!X4o zM`{P*>pV>a2TK$vtL;gLAsg%7Aq!?}dw=~I@!hGqf!Vm(rQdIb>*^%@3)eAVBol0< zekC)yNDwPTx+6D_DXmMgn!?U;zq^*IG1X#NPR#psRPF$!)QPL5+nZ1$zJrYI7?7E~ zl)7s)${JG)-#MDE>JWpAy4W(e?`0vMv!MdJ-QVcECAhH5&AxIhoiW4gGRT4Ardj4h z56{PR`Q;6Tg1V)-zc0J}ccD_o=~m|3eDeyWdl!@ot2TmfvnsZqdjT%{Pr+hzbxpUb z`5T9Tg6KiH{BIJ5e^u%SF;KtLn{W3(yJ7;)+xN1!=eKI{)(x=$b7bo)FHQ3t+s0txNjd2M4!h^iNAfY zb^bb^flN;lZzO!1asQ8UeNmME70TrX^F7(vb=3~lUPMwb4@1bU^i(oZNdFIe-xby5 zwzj+M6+6ADG!X>p(y;&{O$9>-1tIiE4IR-Xy(m?>O7B7-)PPC}9qA7mw60|T4aYgMCbdP2>aH*_s^i%J;RW5kgEBmhk6C|4!uChGW$!0d# zz+Dkq7;BbQpmdV%26LR#bgrjsav5H?qqDPG0~*7~=2f~ z`HAFi0I{#d;20EDa~IXxDX{sE-eJ!t0O(dfwm1joHV#R&&U*U`9KE2qL-1w{Jv7GE zG=x~5-_FpQrB!%iMHGz&28@q}7bB(Ue7+e-khli06uO`4{pADtxq08&9HoNU2fEpZ z@7V2Uu0%+c4jR~YWl~!Wu}lD}DpYQd;Aw_@^Ny8NzYKTV+uGkNIrM=WygpFFvdWuz zpNJ#?gFmD!x#R#TA6{faKu_;{+{Lf!q!H=GiyG`2ku+&UStkAc#f)snzIs#hrJ(O| zWYJOTJ$rPLR)iBI3y?Fm#A5DuR>24Bx}5P-*!xDiT(k#|j-J0h(8#u%lO$I59X6%S zf@}=mB9in3pRH4hjx8LdbjB6CH%5zcm)PmNt<*`+mPHQ61da30!v1%U{BSNoPZUtE zj^=s|_fEtqWz)q*V_~*Ed6mP1EkuqI_&meQ!zX>~RM^mm6NKuWCJTgg&2m#VmT|NV z7mA{6Yu{bE2C!TB_r)a@7ua5C2v@zwVW1JERi5B59%AAu+|U(0Ci=oqBC;z2VUE&x z&*VVSW`Y$A&ss7q!#8Rm47&WO1N=mYB@0G&p3g~&0qx$3LGhjQ33hgNxf64QY8-r% zcf4`^x=fak2(r%I3&!`$G?%(6o*SM@5SLrbB!T$cO;W1;! zQ%@p~B>$-&&hn1@#-GrAF%V1r-h0f^gT3aW4DdwR!LSg(G5i@&5K@T@4FmuYv;Z=V+_>y-jh}GJO62G?QtspUSWu{ z=q}oWf818#&!1OKo|D4w+RwXAGtwPun+dL0w3D;Y&R6{RS^rUk`|o`H6psJO*U!!L ze^(|*^kK5#WNwzk=k*w-aKkf$;k(p(k0^@k1DR8EcGf$u*4l$5YA*9vrFOTwiK%*!VlK@9#Hx zG;@LjDbjhcc!&P;wfA89uy3mdDEfF65rgU7s9E2YOD!dOf(nM@`#8Nbt@QrQ{g2*I zWnJV}OuOaLR3R)PW0|L|;ACaGIW_AJnWZ*{u9tWku5-3WrGG}y^r>G9?`%_YjayZ9 zqzE6gYV3SS7T5W;rb9*4=D}FxAAqCbyT+q|fwaRdrp5++9DN$)JwMD&A4ON)atWx0 zg#5OfNW5XevZJGOvVrLga=Fd5nR6L}ikLdj=(_ymy^e@2#*N@2u!XPKt_o2Whb$7f z5RJURDUlf*1b+Ck2BXVY5y2pzED0wj0YIjft(Fy8rR|^ z{OU3Dox>Uf2(EIZeogec=?w4VZ(k3s3nq^>oMNPCPf@;>FYjeZAi}owK=g^;ib!U# zR1S+TcQuHVK+z)cdIien&w8fGaa)tcj1Tpg(z1oc}ALEISc7z)TP<${wvOuzwhn^pymGq+fNG`>}> zV4>=xGDQP}K*XqFNR*^YG7@`T?Lb4!9mlu~k|e)f^|TLXW7P<6d1L5!Eu*>}`oNKZ z(iCfO?yByFfQW&f#Ur6G%kuLkijFMqBl!Xo57-HdoO;S1fk1mKb}UqREf zSYPSt{`{YQ4WPvA?V1X2?#`~?DJnk{BvTR|JY+9C_}6M6_Q1Ud9(;!{&_1&Es)@On z16t|sRTD!FJOG+|*2L(7R%LtD#Bc!HzMw6nc{kYkPV*n=2Oedhm7~e7)Z^WXBx!F! zKs|t25?kz%UmO;@6%0q+#Q*bZO3jT8rDm1>H45o04NFe4hPSMnZNrSF^~-rn*t2#KKxgP|a1292&}4_@gCI z!M|$*n%=(%Ao$nQK>ZHoFJIA|;EXCZFDs~B;om1>J88X8IylJCJ$PNPOC8s4RVQr# z;0WQ~gt_h8A)20Kv+{R-fD#5ftCb*2?-Ong(wW6In!Vw->^Oz=?*#u58_y!$9vkmAzA*ciJc2P& z-1(8hB|&eZ_XDp5Rm@1D^k`$0;H#JEe~El1Co1j2`{Ab=M}D4D7--ac~K6HoTsV zdlxd0QIUsOD0antV;Ujh6_Z%{imh!e?JHWJG@SCgWtWH#mD^_Ie|%pdPDHp3FBj20 z;E-k2NjBpOgwyaiAnJSi^|hyKR=&=2D2B{N;A5j){@?n{Cq%ByPCdpV))2E9#n^-> zzB`qp5pF4D(>426z(8h-T=nZ4(nLj@l=W&SZs-x{uIMr5{q}~(=JlbY!J;+CiStQp zet!vY24aWbdog(5KD{o(x`B7!6quf#)^EY${YZJajgd=F{54^=<^`QdtKmA?>IuO< zekc6&c{r9>dPP+hZ;2iL#N$xHET8_vk3dr9*z#>t z^UfV|2tV5Wq0ckNQzJk9I%C3r8Zx{+=c^I^9P;f<+@WWh#Xg1{pG}|P ze~3KI>~{*Kx(TThS6C==InQRls#HflZ}YqyXYt9*{z`+t}1!KxTV1xg+){d?!%^%z%f zWQgU^Im=bOSYK$Lai`GR?t4Q5TtPDhNxnrnmy3{mK^ZXO1KO${4p{BL#I%)Kes1-t z-BqflXKIBY`~#Vu1MM<9MM*VKDq`OOFRm*cF(BE+Z>l|oqA#fqd6o&{ee68z)ArGE z!H&)q?a}FG54m%8eg0E*ZcEms0%C=#vL!qywyEz!l_Ob&1I9gtp3q$G&>g!B5gmSa2ayi;VrybKG1bBpk-Ri zb*PfG%F=q?Fw2p8t$UiuDwBeG$qLbJWvN4U@^*=|?asmDn676Zce8`0%6Wv>ysU{9 z=pN8kD$9GQWKeWlKT0oo3BgoGD zeEhCMWe&*VN4_AMF{=3lhvLBtRK@Up56J)R(0;x^l~mJgh!5)eZDj5KVK5QA7t>$-_JyRH=B(MR zR*~PXb^S#!5r^+Q`nN;-nYVDCRMQ-WmDK!wWbMDg_18N7-xaPBHTxwBq$LTyCLvJl z(#EO|>wpuX=?1{&HR#JvI|2Xr3!&z{GuH<=_cdMGDQT&l-SKsq_|D2QEhK<@XUPw0f`s@f;l7?RiE13U+@GF zAaz3kQ=qg1FJ#|-KajdXp3o3L^@dq$0tz^=PR5b6raKUAx6#XNi9coU?>%{e1Ej2E zh7xz45ci2BhZU|fL(N4DZuxTcKS!eSA)g!&1^TNy?-!|+)xH{r#=k2FJE?oe^;02S?`nMsaz+pKCK{smFjQZM34&g;P7%EZBD__ z&L8V~%dMfz&KztcE42hVc;%-x59cv59H8rO&xC@BEcygO@mmd9=B=9d@9=FU*-gWp%34q8HO%!c7##{=G{lDT;#oaaplL0!ywJh?J!veybk> z?`CI*A;4WYaa_Qucc@Z-V>#`WeeSM8a=fG?m)`B^|K4r}u*%ltLl!lCn(lqf3@cJNkcbj%FX`5T5=oQ;!7Yt%7M3gG z{Q0@qlp{keP4m2)UeT?)9VR!^;K=QbvnpOsjt}A?Fwz0zH^^O*V7rJ|JHq3>RrV_O zdtFqK_T6A^F24fOgQfduqG@EuO-ULZ%aN zQa6%%%;}iAFv{<;tfG;egHY3wCI`L`3F!HPMd7cyxh7>AUZS>HPLIM2 zm%YF0Bs|k}LsW-Fz3ZMiGQ}|0fY~Z?iz8cak!Y!!X_!@InBLme!#}XeoO-n?FI?Z| zz4?SrMo-mb5Lc-nM8mNl6RPX(>T>%uYNL+>PzTt^;~B8}XdK+fv1#jQ)PTdv&)9^H zp4)m-|E1og@Msu4TkINE{kprG*!C9$4}7kl71K@$V%1h%)D!3DPWU92sp7loJx*)JMeWTS>n3&CLw!F>fiNEB!>KSjMm1gRGBmeH+eMoKEV)&rZ@z#a0nc~_>b6}*f*qid zpp2j`ccFt1pFQNR)Qy)RCiG;&uXq_=t2P>;${ldqZ$Uh%-rwRh(zH=NQ@h}t<#H+? zYEAsa3~npN(SQmG6nwH~j7_YIQVx>4@f_+BG+B)f!1qIAKi&Z%9pMzCz? zoNn5|7Jz4VdB7EZYw<`W%xHYdt8MZDts!scXm4CNe$ZXHfA9(J#>h{&%N`}#e;|1MGCR2ep4#5A8}lc z?q1sVwXZz~)rev_+bGnJ3Dr2`Xbks2x*Y~N4!941%OI@9V>KUGmO&WnO`SCL22Gy^ z)ogKtb_aw@t_EpW{k=#P?c7_ig&*$>AK^pL88DN!>2`ayf&6}Q=b+Z*`i7J^$Fd|r zA$Iv33fI{M*}ib%_Q?wuA84dic8tOEsi{7s_pPestYPKikH?wWqEa*JidN4qsr^v z_svRFMNkgCl~nizVn+%jz^+S^J5mOZ^3RA8!?fpF%_drv0P^T+3&~P;j_m55%!8!y z@!#!h=R?uJ0H#k}=o^E}M0~W;c|O;glerpSTHfhjGh-3N@NpeE%@E9T#|9w*Xjg9o zGJ)5#Cln<3wD6@5a>EPGMU1UA(uvOH^;qP9ir>%6PpZ7h^}8B3TB>hV-1cpf{#$;@ zZW<=(9O_$|2w5=N$~J5HxjO{ss6lXwCaG%dOTx@{O(<+u7i_&am~)L$u0i6h3WN31 zhpwum4M{qKHD;;gw7GtrehtRxG15_*xot9(s}IP2lFe3oEaFqO(z%KYC&PN@l*Lwz zXV=5%DV-;$%`zi#h>huZrQlfQstJiwM}JMYE#0&4ajivxI`+NSBrN%>=c<&G`Eb}2 zIA>THOeE>4Wu%~m=Au`RZxi1{l*adBl(-FS3lTkk_Z*-Ta0`Aaybzjz7}i|k?Bs=p zN-(Ym7=%zole@2D;`4@^X`*o!k$m$f!KlpA;tA%8wq?!n(}z3k{pg_^J$@vXh)BL}jKG+*8b84b2y#lQd~y;~$<5 z**I&B4*-00r4L)h9X#N=lwP`%q&e=Ch^R7}b1_>lPUmxU#^u7w7VG8fP}Qu+^dNwF z5rcE@$n~l|&3%I}TWQNX3@|w<9qDz56oTv8nuX44lhV2zn`Lc+oug!yCv#OR=j`f` z%ckk3x_uZvsLdC-L4T`m-MIczJNp=T^O%NJmSebSB;UOeE8M7WpXBBczW4)cODww} z2E%&136+#EFU)v>SnrN@#E`nu_ZCmY(v70qj<541RKvSV2UZ6;=lT@U%LpEOQ8M9K`mfRF5JdsFQ+ z9*8>`e;}#GsJYgGkL`hz_PD8fjG<-4kEe&zL{|sC*_;dYOWkY%{WJZme_%1r{j(ny z)J#52+IKo73#)n6RDP^+*P=})t4v|P%gk(g|kJz+sfVPb2xeGfp zh!vUWFUP{!Wbkv?zi9@pmu$Jc%|R__D#S|&gwvOqL*gVz)#$?s!i<7t*~qZK@BJ?A zE$Dp&Mw;^>1pnlS_(1}?^assAs29NMr)bdbr;F@AAcTUoW=U4?sCA4M1EG*rw+{ze zR_e7ljN210&ciL6pW(wK9jxDBNjaF~gA;nOA?KoC^ij?&>D+KIYnXEpX{&N|jR&Uu zPamN#QErUk5s&=fT7NnA8=uYDV2TsIxwD*Bk^9NdM^8b&2}5?bKtlfJse;@}BLP5We^SluQmR&QILyUbgb6{jl5I@pJ|cp%lsooXBqx*wQ0Mdh`V zKUgeq+WU{fQ~E20NwaPbkRQRn>#P0WR2gnLn3p9{H8g{M$@Ap)cJT!FXj6Gu6wE5* z7T?c8cBc#SIDPfd^TQ7~3&pw4O?fg*llTTYYj4VZ$ohWr^OUPB38Ye%ZYyinEn!e> zcx`R6GAOTvrh5SxVZoiQAaM`y4Sd8Z^>!o!>+5ZQ_wo=a^S+0q?^d?bqU$+auhs@$Juyu`d$Y9f)u%>*RMXti_WMI%o4mcv# zCMr)e3);|rNad^fxfW^H;Y{GhsD}C?qnCO^#SnT zdoio15&fw+z&ilCO7Tix$vnA4aXTQqGLNs?2=xxVN}=8lm3ogC8qiivVG(p^(58x~f6}Wl%=^f~5BjgO-<;B{S3@6h4d`PuurBnYjh?eWR=E5YIMGz#G;)ng zr3(k!pK>cB2UqZgy6X51{Gk&ioV?OFP~}N}+lworA3fE^dad8MQZbm|Pk*6>8BgZA zCD|g_+vO64VccGxzOhkz-e5LNs47!~ZzM%XF9bJO7HE4fAFz9~V7O5)=Gz)tT2={7 zF!wjEh!Nn8H_BBpPN_8`ALOhSsMuxN8n3B;5*{eod?14XQ#YwA=>b22t2$tVf<^uA#q7(sXKKJvcwJph&?W zRkh_>VGk=ISbIadm{ho-`3Tt`(&%etXWj7Jg@;XWCYDAtv@(yvU@7GFVn_4-B$Dof z?B|hhLv%?T>1$|nKuFj(EWnfWInAzdsi0Q9oAl9lr=YO%!CasbNg#oBQ_y1`1Pq)2 zp_!3%y7x11@5qU2Jz_9T`hcQr*b^7QhohFToith}86+ zo_3OPl7{BWJL$Fh%=4pUO_oI3IPQC$W@MNWa*yVoD%^7B>J(*Fj_hzzQ7p5x%q1iI27Gq8d>4;h06&A9cMcfv4<2LR>D}7Z?uXM%$_6+F#xLvc ziq-wv?7j`jG0>f)w2Q-Yrzer*;`A5?t$vMP{_2T%H%`j4$d->i^XoUeoEuHRd;SlP z$1g9GYB~n;EVLs;F70sQ?DPcwYIXO`F zjzTX;6aEr-403*K-p!!_QH075PlIN>F8ob9YN%#r*yG4HonD_Ker-6gw5ClwBJc>K z(?!dY^}aQSxqj)@!DlZ)WuQ1dXe@$}8h{;y)VWCp`phfUu44Yj(NVJE?JiDu3V0Jw zQwU331EIhYRMKk7ST)kBy~W}z(6sbUYS%q|Mmbm@0V3sDQn3q}Q1IXmE2C(ZW@#8Tlfg?ieObK#s%BJ5}7xhs0UR%~zJ zRY<_`p4}$;HXTkg=tQ~mAy3CGt=77O0#o80c_r4q5!V(YR=tBHHm+M1yeQYS}@8=uOROcrA-+Zqn9n!9tNiZx^4t?`rfq+V!d?jy8v!(SDfUW!mK;+ErJ1`^>{~$t z&9a4>ncie=eY>@fXRSKkuvMMdb`%_NnF;7ID9mBb<{RlQ+|GRWxm7lfO4PhU#dfNV zWxSbKw2pDy>4fv}rMSYc%QpaQ#>sD;H~uAtvFh)V&nNJ>-xS-bN?1VX4N$3VPu8$v zT?UDGen;_D-}tI3IBS*r7T$%VN@@hpJ44+>F%u;jFcPFl(`BDl9)hdDcylzREIpyX zvjmAXLYuqdx#uHUHmxx=I(7!;Wvxba$YkKvv%6rr4al<~saHsS!%JFjm(*gR{~9Mi-}C~G-o-}kL59w(YBouK7^MO3eoX8Bi zCVpKTsTVYdO)%@u&TXQ!zYB;7Shhe?X;q86ql#d`Gp8iusFMRD4SGJ^15}JJk6fcw z7F)k-uYKud{PuLXd3>jusIc?u&e)*GDhRQdJ2sk@^)j?3?}Qei(7leiW}-d=;TEMbbgRj6m(Ik|abArvN%mYihKy|q?PVq~~^ zz>8U}qSh(P9QD$Yulj%ep05v^m&&5Uzb3Kx0E!q zOuLDdR%xiUg8aPI=n(oT{upXyLP(&uY-CjwU9Km5(am`xhZfiid6jJ3;d*XQH@{{2 z>2I2Gqd{@ez{iGVY8BvhyhN5GSx`^2un-eOqbsCarzSh%s;SAH96&m-ZafBgN`w@E z`GrQgmx%&b&Rc~5%(Y@hg)NJKLD+I?OV#Hi3p8fkfqnKtEk`1EYCbxfe|mZ05vetzp~m5k!|| zYiW=&&&fB-z|E=7FfOS_(0-~-K>oW@%$hwYt26jvy#S~V)(KJW;}%+uW1ac{*RnY{ zwjHVzgj($vGGfJ>yMF&%Q5|95%ak!f#6tnGuNVypQO%#B;E6)irY^{}>RafW89_iO zd24MneRrj=LqPR&I6AW?+K`NIC&c)Dmi^=)On4>Z(1|D|L+4tx3I^Z+Rh&x_M#`N( zCA{X%Y|_Jm>@lC2-R`dmQIZfr4qB!FSXO#O)ufxA@O39vx%TC4J>o3B7#sX_cs^tb zp5vb-I4E+8&c+xbVb;}HG+e*IZiMf*i2pE61;>q%*(GxXVD!vs86Cd2&&CI$d?1m1Y=>8fSCaQd{mo+^ZFJ*3JRj0^(AY$yBW9` zRR)Y?@^RhGzJ;R+VN>=s%gLw^nt}lk+p@z`xl?(MKX0;5Jkpzfvr~QPWclh)7LM$9 z;M)cp8BwCIW%jR$>InXkJR{gdAt+k3LQPF}9H=X-7>OuWKc0u5#tM5VPr8HTe6*@s$Z$m1Mh7 zOc$`m^hS0w6M*V0UuxKR#_fM1r(Dr^_tqo5%DAp<7Y@8s3OOlcb~?qZoNNzx@caMG zEdC)o|I5tck07-4;?NSfOf3tn%YRA7?$fwPe>>TruKHR{wBPx>*quUokmke%(E)Q8 zg^`tNRv7@zSUK5iV^Y^zW7_pE;RHhSPTIrvbyVP`_T157ijk)3W4*gUB!oS>{Do}L z{O)rXHHJM)jFy0HU?nD16Z%aZAW*&=o(5 zwfsG_gyJEQQQuZT2PafwJ9Q3_wQDo;h=96$QZmb0M|y^xv3@@LZ8S<`#GkPSSi7Qt z%4@EONA7T8zCTs@txRMe%{7bq`YsJ;RcRI(eDB2b9s@jTq*G1LM@3mw0#<7AGpB9G zMesj5^vA@5BFsD`iAve$2JtExnKTEEoC*ws=oY7r>sU`Y`px3|m$r2i&E+UEKYpp5 zjR}8BTPF1D1nzJnc--1g9!W(nU`o6(`!T^_?cNrRC znTf4$!4xf1B#2DO8xw?Hq{o3HEOst@GCA9klU-TP04j3q2EKp38sMdAi3wiow{344 zdHpiT`}50)0FnjE;I{U;XffT-03iI60Z7!%G#;mA&(>1qc!!~N=t#4u+xZ~zg}arj zZ#c3q==DMszzyC~Hu=xwR0w#Uq>jQ7BFQ9ZyY=AmboOf=WN2jm zB3g0)`_Rf|BA?HoNub+zVMNboKK_9uZVs+2$pO(Lwky`#_gh8uI8Jb*7nvZ9&BQ=a zcXGhB45OLO%$PZw60?a?H_KZ20BuPc!NS?9$knvzS}MMqBU=i(`>Eu)u^d)J%d)lM z?YumBZt7?%GP&f7!I9XOV0wa;eIVIgU|`rFNyAsoJHf%4qMC!0i%3PX(ezgzjA zA^=bfs##6F+v z)kmaf_?hJt3vb}(2R{yyQjfJ@#p3oDS2T00aN&Eg!n_FvLX}XtmBO6ivfR0NlUr7u zkJN$^Dppo9H*4azuFCkUuMN<`%1jcs^#oS)ID6lg4+?bmhbofAAjM-PNGeYd7!}7U zVxJoir`&R~;Z&Y#OWYe>6OHM`mLrVbAZ?fn(PQezT?HU*&uCc`SU|8jz)?QC&Qa&g zXy5aI4E83jy;YW;@#SGMUs}r|L5>Psm*}~aeF3W)*SIP+qiziS$rNG|v`x>EwVXB? zRm4>~`_Z$?ILbwFp1O2%5#*qC=2e+!-Ym4l=GoA_?Wj-~EVsfI6dDB<*ArH>ge=$U zT1HZoDm1vB?Bw|I+!5hHdD)#HWm_NWnnvqwnzyx5yNuo?qU-zb)F^_KwL1{x__TMB1d& zFTA--FCC>VeN+12gG442{8q#{&Ex$V=bMNAykLFN^Wrv9ysp=9b+)(SJTZT1T@(8Yk zQ^@y|ILuaC3QUb~o{Gwt6t`^sPgqo!=mvniS41*#fff5s?Ln<*D%=Q%6{x8+-%D7G3YL9xjNGp{bqqTZ>dB;tB)o^xvEOGOE0z%n>m;zSKF zMTCZSzEQQ%>rXC2DP0NnA&xc?ed(IwY5SmqTiDqZHTeiQ1iMvkSs=FUa%tUwNW*S? z6)G++-V|DHk~k?ZZ2{uPE{1dck@oIw69#uU6qRF+3L@DUht!bf!m~QGf?Mo^?%P?| zmbs92$k$}~3GxqUP&jip!V-5YX5_asEA@v*>V>0`CDM5ltMrCv7aoaUZj z-tttA5yl#B@e`h`!;tlDRu41yB;)zpA#joAS1XachWDlnzsO<(=|wNcyV|>BAH=$; z31zKztnldwb(kui}!rga4ojFBlO!^fS&jixMt>)F5V~Q0EjITQ9!v-xUlJwj(*f*AnqLnZim>RN~SRG=?1{4bhFgjrvd3lYqynFHZctZqY09Dtw?HrWj^W z`#zAGSS(Db#_urOA?#4Ej;4C?Vy?%I$4;|9R`TyE!;&>k5U$i3+Q<|#h7^6K)Udm zZdgy6ElPm}S1B0OlgrS>T($D<-9=gl==bJPF}-)6mhAP=wsU7WjALVSU;eTG=*8#% z@y~PG{gydA(|v`W32)La*r6t#5P)|cp$OSV-V2ZZ;~|IGNY%o}3I3vvEVFLi;juA` z?)lm73NP(ar_MO|W{JysFt*dg0j*5Mo*;8NMA!y;x9zE%;Z&g?T~kcF^XhD#>y%Gl z!9e~*NHn{x+GbH#c1n6@1dabR#hEk41i+k<^R)48Yn;ty6#s&}#um{uO_xYEHCU0- z;gImbWFE#1h~&E5M0}Y>(8`lVxHpoeb~A=A)pR5u#V54y!%y+`tQOkM$Pm)V69}hg z3k)D$?*2`KqR7YsCFSTT>!3ct-aMm1v*78^=mVa;@VJGs{mzQr@|{_a-$%@*M5rAW zCC?Er!q3d1B7D!I+Z{I+$J}S#rXCuySZXWty|Ok>Qy&s=o~sYzXttmI%|w!B_zeC2 zOAXf(Mtn{?dU{5gv?Yd3=b$cCD^NvVePk+1pE(fDZopa>%~qkBuHLlzA(N>k!v@t*I*DTD@M66TO*;avJ03T&MJ$z0Il*P#wGks=MugHHCY9*SfV_4VUZ;-heDsSqrm)a=iOTaV@)4WnU6N#ZEv;JP?kh<% zxLKzEEZC&=_?~k@3{st1n$UicudK<+-FXM1k&kYFUN|okZ2~$;6(!;NU3{iHrU^ zltWhFdq!tswhjA5;mII>LKm*=^J)S%gg-l^;}Aur&=mDjccFJJ7(*&+`dSqaxF zwA5s?Na#%QqvU>FYy<5f7&XNirEqlBtB|GH`D&9vUKT#S9ecTlhd$S{Z>p^94yt@j zzbxzHqXvhC;of|PD^ z-~RF0_$1*L)u}z+ee@0rwCsp(W7?}>)%ATZTB%lFU8mba9M5<=ftIO4NwRx3e6s&w z0?+J;CpUgSLEvy-0WBZE?gj1Hu=3zh8G|0mFVFt7zWbLpd~y2FQJI8+P=>R=*N&ww zdLBFDU-Oas&!OM0BIub^$B8N*g$SPzmav1zhUGKWM^oP{d?4FvX79v)!j{5E$D6c< z;bYTG@QtsJRc}8~R#skES_%<`zhGFu*}ta{9ToGH0)Q&#B;EG6&MrQvZ1oC%ZkjZi zr6qJsnsJPFm|?HQa4_F@?OR9VLqvltmI0^;%f+g`5?8tLmwqR@q(Q45H8xtmA@451 zPZi*Mv+c-U!@c;O8O(fqqC|_#UYpJMEZCJ4Mhq|Z+Cs_?pFdufn_e>CYn0v;KsJq^ zj9=PIkp3TblH>Zj`Dg3a^)7PJT^xVj9lj%%c&qs8qV}B2ISz{L^MVIY{d#;_9;>p- z5D55Havh{DTJ{&Um1)oxKii7o)aKpx$ao{#_(nX1Sw37lm|5xaMDzN+`q8wfjj3|M zyoRRGBS3En9b!5R_+@5dC$_R;V}z~c9naU!^%pC;ZI2YX&+sh!?VkKI0sA*6kgeWL z3A=Wf$M%Qkhod?(P?uJwQRFN+lx#}c)S}}jXno@(-60TS$;0M2QKp3vw%2juhwpy* z(F0(9;8auUr0-oOMtVum%kW> z6Qek_`3E5a<(9X~7RRnOWMf4Q>+=_o(|O7{H_zF!-E~^txWE4Z5tpAI#!=47{xC6= zcm!Yn)~d34aB%QTvhoBAZeaUInnv)fEr?Sw9epR(c`lweWT*N`6e$LBPFp& z9I;hOF6?SRW2C4~l#qpC3yZ?*`!)WX+D`ZxRyQ{_mU)}?B+FROU>12nl|xRD8f7~6 z6s7oli-ODfHYp6N8sMTOKYjYNv#$H2Y<0zl%*vMvVc1}}CckZ*-dZPKkSfJT7=iBNFs1dG&U2K7D<*_LP>Xg9xk@GG$b`@rhaf)87?{M4(Ck{3YZBLpih1HKWL_Gbn z_6`1(b|q>C@_qfw>i6$&ZfB<$COXQJX`N>1gSG_G@I0^{uN^ZFVK*>pbsmHbihbch z`;4%^I)O4Cl*_Vp5m~K)2?=$k`eaJCCDxf&EDTo*;jBgCKmT$?ero6Vq*|gp4@cMT z++W|QFBrF%B#=o5F-o%9(Wr*4>$M<{E7DYOK$|z}PfKpKFr1#<_5s9S$}e8*H~fk+ zMc2S8402X*gE};V>FnJw?TC)#X@o=W!?s*kIIS<>?D(`yh~m`P2(ZB9Nox_P2XewB zw=Z5Bgqd@jxfe`7dUGP8S-!g&WqxI2q<%h82Irf+-!2DNG3IQ+VrIlhIHh^EZ>X}X zZJNkx9}@_DS|3_HXGw7O8wn+D+7LA(1r35XExL#<1*(p;z60g1S|WBPOjsxT*Uz8- zQD^=_14Yq``#N-B(J5<@7dmj#S>9(j$#9a5XgD;SGrIuxk-}Td#XM~5somPf#lt7= z^U%Dkp!4+HwJ*i4eAk{UTUGsV=a}Kzw_N>u^f;@$#5&wWRh>biY8r1azsW=A&CVCa z)??G;qPT2`w5eSpL}r@8CFO2;*&*fxxRJAEUEztwj*`Ns8;y0=3&W1Z&lX-J%2Vcj zXcu&;&FM&ch@lWhOoTs(S9m>I_Yxf;paMzLMJ1z<7}Y!mk+Q)#!uopel(|l6zzlWw z%JnL@)(GF;5fs6`!Qe?hI5&5N_AUNz0gXt*1{sX`nDU@}LSd9#p0nkf z;t0W=TLavE$`OBjo@`%n<Z=s3TBqmKNK7-aD%=bb@yz=Abt9hOY8i>R5Gsi##B(`*zy1QLh(rWQfVwp2Okdrugt5gr2iJoev;DDHG z(Wm2|Hq&Vn(+C@xw1t+wYECfmP>_1p*K)*tDk?$b+T7BimjOQ#QdS%Vlv& z%5OnCyUcoR@@A=CUR4IZ>j0Uq5CU4buPs%>qP6}WHeWygGe*GTQR?BhBG@@gKAlX6 z1}pE3Svc3DE{K&AwDt;0l7?AUp+XC3nTKbUqob>zzG)uxl>>3(>U{y#P7@2@bu+B0 z{R|vE`_&3J*N@jX>tK>xFFdVbl6mp_ z;F+TB3bAe4`AV(bbF*!-PlvEr87Q9E!C_d1;Ye6p*AaLW%Ki3_rPsEz{W2o?^G1X2 z8~$|QUQ!9W~LUxvw4x%vq6|Y6tG2?T5Y`qAg;*`}+RI z3d3aswEIE*mhl#kXOZ`Nc6!Ira(@pP+^C0tX-`EVA)f667^8=L;HZ!T2MhS`2r zJ($%_leTti1EXw2?1(-a*t`Dmnf)qbk3r68eI({^@d&F6jgq|5FZR9Iwf*n6&kv>; zj{2Y9bs#t#J_O0qWgp~>fu>)Xzq>jsob>oJrJ~)a05zdErETkFbMQsC{iTAb+2kUo z%lA1{mjX&y-zuUd`z zq;bHuD)sX@s!8sO&5|z=OVco;^w6}X;!n;SaD^*QXw@)F{)X6^;0WIfA^9e4b!^EG zZ3(MQtRI(O-+HBcflQNhWb3(#%PYl7u#-Igm8!A4LJ4rz)=%xZChj;9BtI_b9!c$z z+IoSETG@PM)0O~dQ^8GhM{wzdzDe-vVgD0V#{2ebmwa~g)mB|%NsquIMiHZ9!tiXz z6X+4#gdXkI8M0u3Ke*J8553LkcFWjNvDp2&4-*wA7u|{C-x*9$Z4F`0Gx=iAd~WYN zQh>9V?aI5(V6|-R+|Q|^mEN>MgCcy2ef{Gw8H0_v9utvs=+qu4slUXy9(}pCp4dgkLL-5wZJn_`{O| zxe(T@wP*oIjyCGt^kQWV-L&Tsr9>f*jKP%|Bro1Wp6(Z`T5O-k)4GSX!h?K4%THgl z%Ic4>?(X$2VfbN7d0Tp*kI6*d6Dp~=7U#9=lK5q2NFQ6dfSrsR6+83K2ko}ZK4<~! zx1`#UVJ{2gPARZy^Fx-T_U4ez^?mEz2D3N6|ngmY?7nPbW=C^e94(1~1wlxOcRFJjvs1y2-juBvXi9KtR5 zabF1Q*2^%&BaBvayEO}Eje`u(fWXF5l4+XBWH_;HrE*{x7yF=GXBkn}U6!>pP#XyN z=Aq;Ibs?hhs6vIQMSHqFXu(y1CefC&=(kSp{P!tbi7zc8B(frwXTJ6Dom~5Vm6t(dYcS~|$cTADvV29-2 zIm!k#h@f}%k`O;q$95YX$x|^$mm}Efwk$k7Iug=8(20^~Ow-PdLL4bbC(5`I$~P43 z=vM0Pq&-S-^DbZ3*;u7vKCp2L?v~Q5)MF)oF;FYZ#Q%3E0dVU*IuaHfs`27LQEmBR zl7ZTo*MJ#gpUdo;GGiaBd~znqvZ59+dTSy2q>ZYT5j~EJl>?|T<2P*GEU`$r^lr`x zvF2Eg96$-EW#fI##DgWriQcG#_7@r3Cn>Jb|x7z>(%*^ z^SwONOSA%67BIy)v80YvU36eWcek2Stmf4QcWu)glhn78di&NV7NwnMV|hZvU@yMd zPN7DpKW7Brc%bjYDCt4=5ablFsiY_e%WrGHKH#;Afm`06wv4gu*7L8QB+r5hxM4pHfnmKcy`=pli>i3rvT?P`~~9as^CIJ#uucdG6ETA_&`gxO<) zp;%Ti3j(fH)Ec+#+$26{wdwLrS~}&d2%f_%=|F?b<@B99%YdIANoDCor&=0PKb8>- zq)oLR;Ltu}@X}-gcJ!#+7|Tv|)K!TH{KKxp0en26raY6a>(079EomzZ42^Q5Mv~Kx zE1?M3#MSb_N)M~SBsanH;%uv#dcm|sJ4Rw)8+`kQJM>>BXrS_4U22YnK)1jmU6uLl z&^?1ZwfQ0}P&uPsW$AK{mN2!K;ItimuY5tV)uofeRQWmckT9Em+DEL%kJqi-+dsP1 zQpqHBy6$`h--o;Gxw&*i5G;KP7oS#8fPZGSPMfL0Ufz<+tN5pk$v?4Dgu_85=@iw)q(lcBayk$C5 zoov5!fo4h;$E?Q+$CNMKjG2R~Sl&~wO6K!cvzc$Cvh==IZ4V=(Lm;Y*E^sAIWYGCU#~md?Q1689?rY!L9qtHjJ1hb|w^D7kn1DO}M&JqKbH3 zV$ik1@D&&0u;xkk;l3BWgUK^?pKaIk`BP~l*hych!-Nx&(nP^wEX0kGl13g}tY2~B z3RK}F3fe>fMVTlSzfsmQY>Z^awPdA@m3U4bdv-9Tgg<$E&3uh%vx@hxW7$OiW#oHp z9k5pQ1=3}E78b1gFXNvwnX)U-p|e}6lL|OV2$Ie)v~o;*m@HZPTp^vEP(g1fT^Gif z6!Y110E{`nZTB+!{aC>#jc}DJ!;bJN`3b$n*@PjBM6s(}I?d~yYz&9s!*YP$!?P9!zr~-HBZ+kSntt?O(vrt77 z19?s!7&CNj)v#qaHFO`GhFSMR3$<&W?N-=4^Sp4zKK9Bsmxd^9aA=9;j=+GJQfW95 z^3IzTHv7@T3p;9 zkk@6p2)eBC3M~s#NJ)RO@L8Uwd&Qec61eNAd^EI_tDsraBRvcs>W>UqG-|hq+-+uZ z2Kv~`yE2t0FqSLhgH)mS?%nH5sA!tbZIdIZ?HM)4mU*Mzr(&jCHS%@S0XAmy1U2qi zamV6u>PV|!l^FXgjBxIL#DHG?lNFib$IM>*=T_g$us?E^Dfb}s=W2u-MGj1^UfHkq znok+g3rgcKGXgw-vPByJUd&vDRjlOMuEuk4F-vYJt;lx9s|gZtOd=KN$R_d~v;^&L zP3Lpy7Ez#WazyURwwYAq7C|>G0I!;wi0jHYPwII3MHwDMo|9~<&Q{)~2xs+tExE^b zAmv1W?)(s+^=hT!3f6`2^p8J&r7Q&xV>CV(-DkBKHBwe3tXJEv>cNwW!*HLnW4fCE z{v7TkaDJE@b0Ak1t28CkP-i*7M8J38z_+?1z*o&_C=>~Jmk}S+2>#8neYA4h1T3jz zISg{o?s7Mo?VY4nNnCK#BpkPWwlCWZ5kg&MbkKz({EaIgl#86xb3X2ir>HXO+cYfl zHpv0^mNIZ^KUr1e|iLBj;si;j!_P{I^Bkvsl;S84oXRh7~H@LkTW(FX(cF?rH;l{)z?#>zfC z8*`euV!`!R@X1=w;H{}TWs|;m6VWl(ve?NAZhS^Ww_g{)p?DsX)|Bw(h>=a!R+$LV z9}mwYIT0CI9g8%!7S5-+ye#aR9C$Tp8K(Dw`pww(&g_DGhik{$#C&gs{z@|JEA0iY3P*`~zm;fqN zdB->2FSP~|?UWI3C;JWSNvVJFt+qvcM^wi(9A zzs6=rRDaGY{5k4*+3?O=_?^{Mtz_0fwNQiIsE{uy1r`vDNcda+`P@JG<}ZDkFN;iH zy8Z-oe=&0`t< z&nT+{xG>LWf;Rs%CzbsINHv+C#XkO@8B_jP068?=7dTJErjfl^zP}$G`U~4>scI}U!O%t;9&fqVgBRvWT1jd48U(3!=MiPt0KAl5W75Ebb`kwtja1%@Y4F20| zbVQBgz(eNK`f?7Fo%qF-WPESK9e4fJMuKZ?gVn}&?#_E$tI2Vd?Y#nQK;ORG0_WbH zY5L&J#Nwd$jt^M$)DM6RXsfL+%fCmp{4uvRSdDRQcSGb?Ca+;1!VdMkU(8|r+YJSD zUw5M+b06AA_58W;t56aX|H1?i^2B0vh!&3R^*o6dW`ee#iOp?Mjf9y%EH;rXSD6tx@qoXY{$7Cnszqcakmnp z3W|xWtss(0DBmzTTx~LH--76emFz=zrD|Al!2{#q{#n6KH$FdrWp)9smJ~I%3gYS% zFJ90gQoH)?*Gg-JqKJC63O^SYm+pDKs?I&kJEBL574{&AZb9rr;u$`f`A4B--0M)& z0PQ9(*=c7BDj;f#yK!vKvY*1e7h;hy1eRJO-ZeKPL*KI4-&P5E)gP$#Z|hz-8YP72 z*jMJNU8p{ zexTpwW8kf>`U+=K_QeHk*X6EUcyW36s--xD1*4c?cnEIj&Nq@W%! zTR4e)KXDCN1?r9i<`;v}yB~D^&2xOhi-b-x$E??%^lZ0k1LwOP5MPmgclkDHZoise zv^CoV{RkqVdQK}b?c_2sl?7PYe$S3DKi@!Em+1;|_o*`H?90kkbplS8@&eAjHzK{< zcKjLQ0~jmo`z^z@#^TBZ8lnarC!c_%KC#Yc1&&KW^sRY@L{8gJu6hHX0+&pRb$SP=a+DG|0EJzR~hJ!qM^sAI;UH7{E9>? z6ZSTV-OCTha|735TN&kf>Sn2Na0nC~n;OIZ1b;j%Y^CpBDAflPJ@x!cU9WSHSdIx0 zFs)R2zdO9l%j}jMqvz;#xV_w?q1OqpC@JN4JOGT`bE@9>!}_V~LbF#qeC#2S8JF#J z_Z&1=j4ximwAPAC^_x{VfcIUjJIw_COSE~d?_->9+v6~0Q+@2`9!?oe2>vc`lw7hT zi0^K?#`B^ z=5S8hWKE2{VK0kMQ2(IRfu~5uWjh;X^a5c zb1*eYhtMrWxVc)q`Gt#HhL|Vyk6+)62_=cn~{6g~iMUMT< zzrnFUGQ?R_u?@L#ZSkv_(ZY23j+qFDkVb`@an9;Wk6NsUD{5!!fVh9{s+&k2A&6u| z)&wkv#*R%bWtK~|Tcqx~I->#Q*Z=;qu*!GRyEyz2J_UC`?_+-tex+nCpEMib*?12X zD|nq(u_R-k#q&6e^E%s_Ld{f-!S_z{j4BS4L`8M@%3b!AvJT2Vk9r<*)D^CEr+9tp zj>(O;80k8hBW-w}=djd%98EN_dw=dLEbkRD@1|R^YjTfi$a`Vz-)aHCd7KgX1#e_O zJxOJ^P@5?9ywE+YRH|LhJ8^Sqn#-T>RB7LdV44;Bgp`#4`g*y$#oicpLc&`W8ATH= zxf_0O8sWS%&6FZfN<$$oQ=AfZtWL6h?x^?f&8R2a&YwNPpL`gE>aBvb(S^fA-$M}r z8i1crDWW(*u7g;*7SVA|8r*Q~2upr8(HmWMr!Tsq_U_$#l2+mHr*AH+?Y7TOeu>NxAlc^Y|IQ*ZKM&g4M%kn>=KS!h~8nQv#Npa7Egj9!+AD^CBz zE9oWYO+U9}7!NRbfGU8xxtF=hW9N=sAvS{2{ekBOZutt~_YO{L`IYoEk3Mx#HBEa(g0=-60CRGC5AzsS4iX=?FlK4CLEA%EjA8D-1 zHYGkUHpCbz(W!BhEH&=iA$D_0nc-Y_Ohz_bgZY+;c@t|LnpjinHHw6Z)@4}&%CT+7 zw*Dw4KIa{zX^5xu)+Du_H`|1(SQAEpy>T$95ByqE_=rC|`%fP8&mZM|-xcB}O$RnKYx|AyvZxtO<#h`_w3ie4S7VhMqp5Xf+!}2%9PUf)p#9S(s*9N_LiUJi zyCoB|tN<~-hrBsA`df-uDodf?ktj*x@lbZIil1on8xt--$X8VaHOGdVBKis2*do_v;R z2}glJ3st>1XjYI)9xIp4MNgr0?u4#KYR;L_VML1wSkbw%O}Z)e#&%jKhM9!Wtwg0R z840JPVmjF^esk>t^A+ufGGu#%_ta$taVWRNHzK{n)cdW&}Fw*W|G3W$xcFSngV3q)~U0n>=VSQj!u{ zJVD&_UZy!03b3eC!Vbga(Tu&d%A|7D3iZ8h=1<}YlXvaXJK>BmW_Of{{zASs?d)r~_*?2m4S6&4NA!uT!!>!;l}Ce>H^j~ z@KN9F21RRV1V3`#^(aKS#}^e}-0alx+t?Mz<~dhCk1 zoo5J;mRe7HApr@#)mf^=K0XM}PiS^e-`@ekn(r`}Wu6Iz*@=v{lnH{m|2)|>!`YaA z_J1TrT@=1JXO?i{j2-d}FXN&~PNm5|DNW4@lgM88d=7lObCkM0JNs=@C|^MZ-rzt( zt3sQ*HoGAT^`E#7CGuL-9f6aVLDKnBb52~(55%fO6dTq?AyFUG9_)W6Qr-}xep2

#ooL z8G@3RYsPf+!?)k;)czm`CY&hQ7OEO4u@s8i{%k2!!HgzKk_ZzBbMUyyAInSK@G-j% zwg1`MPLpV!{iI9=LuPD2<*DeKZzMXR7*4F5*i=MQiQCl_t}x43F#~9?Lr;P9FiAK6 zoqr4?eIy=mi^y4%>T+aTQe{`dxkTi)PZcX)Fz^Ma-(%hv*oM{hrs3O*ppwX4$vARmZBxky*PDQlJlK%s?=Vz;9BWL;0s3 zEtK4KQzwLJ8!9ufb5H_2hq`U9dMUYvb?a^?M}HiRjW(&)EF#l@3(UXr ztnj#QQCho-ATb1}9>W=S4qae|6k`5zz16;yr*QPGlB;N1HL1;U8(Q<>tf;=k>qvsK zY(JiMfMv+A(yihssO8M-3>Rj)a(HpD#z1MyWknw^qi}vYuLY3lB<%s35w!9}!>a?$ zF~aYQi*(+*GvqDe4<8PRk9Ch_x>abhzJ0s<+*6d@YvGG+%2WHR3l8&nNX)}^K5d(s zy{N?$zpZN@Epq;&5Nz?=x9?adM$;+vbjr8$K~s1^+G<4_R6<1;fRxO$vMhp#&seVj z5apCdZF2PZ*1~$|^t!l$8zWbQ=B^7xIf52x0FM3{e(TvfcnEs3kb0-zeTHCZFV#?nmYC9Vyf9KJvs)@H$aLD3_h>^C zF^|0j;sS;NQD}_3;bz}^xVhmG_Jbury&!8qPwdPsC;8@G)9miNX)y|wEq8%Zwj9N* zpanCKsWZ7_4r)3Ypt=D;W-b=8LP)M5(R4bx09TEP5a3e+FbMoSrztFjo0>}&2>P^k zl3b0J15kz`(nLIO-=^GXkcR;jh|kG}%NyI2O8i6{prmpgdd@}8U%$S}2@f3gykJ`E z2M4M^KTij|VbAhn3qk#d&F-!8leBI*z@AhA_Bhz@%SUh_{T9U|&)fd!yK&5>a^u(R>L)w%jMh-c=;kj^%%O`qril+$^0-e)_z3YfqjPlJ%i{o1 zcekyjSB(A#aj3<}Lu!do><&@4qf3P_QlV!BT2AaS5ge3GRv0x7G@V1fFGo9kuH21B zR?%2@VFavpGY{7UY7d=Ss3Qo|)q*W>mlvLT8Vxk+ln&JE;VU!3w!uRxX?PK3Sr01H zQSN|bx}s>RmWB``*~gJ;jL@!imX_t*c~$JRzk^XqN4~jz{&F~`w6cAyX**h?T8rKx zxnwFa6sOBFAsF)ub}o}vWoPyFbNz!Srr!G7>#ygm24Axpb<#4ZdEM~W$W_?9uc7u( zp=MVE&`O3y4B;xzt%H?(=H)#g$4}!d2+_%Fw3;jZ`mtGcx!LE6^)!zllE*>Ke6BpE zK|PP#bReDyp9x&mG0%KGPVmu-?V8Ws`}NnHdkwe@_0%OY5{+gX6DI^=U(Hmv@-i7Z z_BwoQ7CNEjiaJbJYD1qJfWdP2JLBLB1DWRxLDp!EcUGz8p&YgK>XSdK=RhJne6} zK%i)xt>$O(cyn?FEH`0CmV(SLeLEC>MQhHwVh4*ENSUN4yZ;fLfo$t#9yVF!hFb(c zqKs0cMjEyLmceq;VV5EYa^Ifj=tw^bI^jUFpupH$7Kmv=(h%+ml}?mnuBs|G>tXW-AMSD^z`(~IMmdHSNO z?=GkG;9G0$hQu$sV!LHY?^3B_=*F~WS*YF_ghZ`%0>u%HHjgBo2xsq@Sm=7L-8-I( zCE_ZQUVP6?n8#2Vjzb6dmBC+cr6&(|j{>yKX-gPFo8cn;tHn#L;a16&vJz#=FjH+F zyYB&xW)ov7CwM<#ovPlvmF@CcAYm29TQL`HBC8ot6yNx|WiA4sHfw?yByTj=s82`(oy8z#AW7|8K%K74nUS$E=kVIcoWO)-Lagn8hk7ewDkZ zOsVQ4gz4m1fa;=Z|2rykeDQ@CkK0l4az9x80fAn}Rl0ac-0Kql<;i_f6U{jckr2Mt zJIT$rzOG{nuSNX5#ox0+<^Cq8`WD=MGN070mR&c7;E>R1wDfs4$+u7D|m)#$9n$?_@4L9CIuSY07gITT1>LZu*diD4&gY zFttAX7>Hdnhi7p8RZv7bWZ?;`A@RY!gzFqqP_1k{C(FyR&6#-VUYy#l-WtB)8h1YE zt2}xbdo>xq&Qo@lIn`tqS4k*X58E30hu%`ZQN& zyUo1`k6Gi*xYyM;IidWq#*Go>k(0&!Lggt0)Y9n>&u?DAa1D@pwm$m22P<-q;3`$awG7}{RGWBxrWZ=EUl+T~xNMbr9MWlG05BVib z+X2!USzKd9^XclEheYg}giXp^qlel98`G%(>xJc(iB}B>UI_@p$3D6dt*q&(eQwl} z6pB(eS55z4a2aof)!Q8UEG5wUE0iJ(G%^keV2cX(s9QIMJ_Cd;SxRN^yZS084FUGX z2#8;RpFC{uRS2=sc(@Kl5e~Dp=y2*Caj81FueWkeQY`^%kaO>QnDsnDCu_R0xB-t@ zmw&3*uA#xPLna(wA1`2;hlb#UKgaehIFo6BF!;U9MTg^0W45BMrZ{AG>dw*KJoOM@Gs zA_nfDx!r8RFJta$29_hG4LW%aH~gF09;IO;Q?v~0Fi$i9Y05TwgtYv-AG#_yfK z25Na|@)%?pfe+5Dls#X+DAa9urcrj$ndCaM9bIcak`Yr2nqyx=sX@>6>yRblh%;xS z4o;cpV2jzY;UiA5;%VjkP&G^TJSW@JS$=@~K)-`D&9Kh?IMS#mVV37;OR&azaDyMS zy?+ALG#J5Mx!N}^(dgkz?C&J2u7jvZt7RG7IPrJg)k*O|h>8%e%*BWwS-c z1t)-50ic)A32HceTo;+JZSaKZ0Y2Oke+t09Ph8yKKx3}R87kR`Zc{IsV^+8v@u1;t zu&q0->omnC6WlIKPXS~O@o*_#bi*0b=)385OYK#tNDQSN4cEg@!p`qAfFYz;(8~OD^$=YOhKemyI^Q_qA5qK!JLl&og%Aj7>WiPqSjYIlQ6A6%U4o zuFr$cHt~dOjTc-cT|CzFoEl?P3 z-i=cHkeRNrrkp){rQhc4WIaw;G{IsS!Wp^Sq(?rM+i3It8`LRxIy4bcs+#Gx{Y4{q zlqNBJYj#cdOM}bCgi@V}02|y~Jf6=x_8|*yqPYz6#%oiyXDO)o{m^6Xr38yHiVk$C zdEF1jCrQjL7MuLeKOjk7-$J=e*yIgaWgZ@bRU{QgJP==D@F7o*Wa8LiVyDS^f6>-b zJ!V&xMoRP2bf7 z%>e#YMaz6OT{Vt+Q!yVk+JF^;;axzXIZ72`kr(fL)&vhR_U7oTuebHu7a; zs~;HG>CsP)yHndw$G22Iclv>+$1z>7oO160tLA0!yDr)C$iW|6rAlP$gHe>i>`eWu zb|*I#FgfUJy>Y)K;VW}oGv5PMtLF*-3tQDU{}e*XH_F*%o3#UgoaXLZXP7*u?r@t`yaW%Tc$RchoJAamQBz}DxNduYNK zp+)_RZuuDl5j7c)eY1)(!s7iDj<9uzyIlF2b^_HNES&9dujoTDIY)v-#B0fF>lmtZNy}Bb7{XU`>(^qsO zG~Ih&(~Vn;7vPQ!Lsauj&vy@M%TfTjQT#^Ip){Z&TG`I}I`X*AYoZYDHZsfS9NHts z=DB;1&e%h`QR&rDU9hB9J`UZ8?`8J3&rs&E6tjBa3^Uksp;W*FBEgbqfdS8G!eP3M zRH0QDvluU+1gI4(D>Esd@2Vvx)O?hmtdwrlc$(*RCF!MC#XDnZ$1m_uO#`AV)bcV@rNe|Faes4hRg{STW3xv@oY2qInx18v&9evzF#AJU>#J;l<;|pB z{V10R%**9kisRI+HxUfLv8TQhO=>CBGLMCNP3r`i(>y^kah%|@WE5RnwzSGH%dV(w zR-ddUG$WEV(vuB^W4EjL$Jk5j_kG+@tQXrJaBh=ybKVWG=i9Gt*SvfH8#tviui9P`@Q*zVW z{t_ap6vA?^Jug%9@$#4%ln0-H8ZSL*RC?l)^NK@~(MajQ6SrA2%kLUZeNRa;9KOOU zn5*ny{!BmYY4R1SLN_v5Av&PPZHYG|fV0D)lc?gY6wedNjBptGSin40*&!2X`&#RJ zh9s5b_m}!Q7XvBMQJaXgn(Onp&@}wkUIB^!8=}>u_XvaO!R?ba52a<4{r7@h4m{y( zbz88mrMOe5rT?xcFJU3id4r*P*7n2X%>h^PdJ^T=snBx=8pmd-9v1X_Su(bea+zE_bUeXpD6WXpiS}ZMto)wmtf5-|nWE6t1uCmRD|%Up8tM z>NVl*RB)4$I_a*3nJB93oK@MkS=i4~p6FlU|1zD52;SE~#H9mRYqk(y1a|W?d$}+* z7amIT>p+d=f^sS#BctyZ2i$G*Uxng+u)_c%*h4me09N;*iRCBG7z?yVu?v#^uqa`q zB2pP@Zk%OIGsUn_M0KGk@IcsKd1nYgeJ`36*(D zkXoHHuFfMK3mM95OBT3_*WWVG_Kr~g1KY!Ml)nkebwV^T?t#PTEYlg5JuVtqUj2sm z%SQ>o;m-n2MOc1 zI=s1WO=0k}zvUw?m}9+FarDD=ti6|XAqU)}aW~1D2~=jFbrlCq@;@#f12>@Z%~2{v z>aD*;?B!F1U+QN!hFLCb&QQ-IXxG74iSYj^L;6{3@?A zK?QOdo1J88v*$p-!J-vD^m{J`!T`Eu*`dy1)^DT#dy_XJv6}@_n7Fay7O(FM<~{3_ zGln7a9VdI|)hfeH{@6WT+<*cyDapG*5r5J_-pr_cjPS0MA$Tl7vjXyTW3CJ;nm+|$ z#+fQshVtH)|0Xx~BP9b2r_X2FPgKfkDD~L1fQbnSY;n)(c=`N7#`~(C|F1K)XLdU* zNUi}2YLF8^2WIOKnEI*-ZNRJ@egyv~RrF-<*#no%(^#N}L0oh5F9aM;pQ+X_^qIR7 zvxRBHHrN_JRlwA`ZX{)Hpt;Bpg0jvq5-p?Ee2D$3_Id^6|E=VNXu0|M&V8kFGrK_8 z1|0x%$CWIJ2Q^(6JhJ9SW(3lS(;wOXx<;DWXcmxpfn+2uWqt+kv4_ix?2}CW9K-*z zlE9{|hx3?g6Lo*TOJ8q$vV@{v{nEWd4Jg#(Kx+Am=H9soP>7U363~$y?Xry`fyIq) zsFj6QV*3XNv0-2>J}95lD|9kjr`?3-pb8vjHBGluVp^b_1g`vm%kaRF9c&ylRSJs2 zVb+y;^^k;tahly}X7lo4QFw-AR9CcQ8I@vMYvuM=(fdiK)oksHq8* zVqydv?n?tKHZk~hj=MLH{qm>F;|VgQR1YeEZdLOT;Io_)cgQV@h{-f3zI57|UgW0o zxrJ~7cxzv&qw5#TrNF$Qj_{lY9M{y(Xc=Qy!$NTxMuDvMun&pU+LtQb(M-%F)6V#4 z6rf;0^mgSjss~W)qY?q$!6>71KR>H|E7HbjW+*Xt^?=hz_SzG=3njZLquWbBJ+y8H zlS=?^&Fs71M|#XM(CdkmKG3V)7yOrdthtPQ!KM?!M5wumr`rjcm;pDcK`oDS?&bcP zZ#3ijZF7C|6Ib(_VViu9D6#xwz8^*L8sN^-P|m;QYGTtw6belD0Az}lDvqY^;;-q} z73TM_Gui`}o(i~m(TazY#f~%bC3JTZV)0e_-w4yzMaVaonh_1u(aMMaL<3v|+9@slkm6}%S?Vi_x)nV+Bs8(i?Gv)n;89TuS8w1<1Q8Tx`K^GRv zTPZcM$PI`zvDq693O(s>QL6y;?;`A=`rnpc^dr7a50%`MV>Rmhq^S1)m#K;3lh^5U z2X4Hh&PEA5KzX>{oWb1|C|+Q6lC)vJbE)B4V}^3B-^LpM+&rW~tx$&Qh71K!C=ANi zyedN(@c-PH{rS!Ly9}8KO+a#CLLcd$7z7RG)V3YnBf-uviL-`s_T<-$Sdm$4I(^EU zC`_P-uE~D7%wFz}(Wolnm%?fakJ!jfP|dKjV1X~mo{!W*f) zmF(L|=e0EKSG0Y%C|`F|D5FhgA0OX4N%S_Q@Xc!=s@b+eRN+ZO%~8hvLMrhw)rg4b zuyOjKz!DCKYCMBFX^1-c7k12V4_Raism8W}!mBInQqT%_zD>>?gN>e3w_E&fN60`a z3)P{>%tx5}3c+ax#p0P0xccnbK{B|hqh?_qGpdR~hV4*1G_X$kjC7i4rew7FRc+{T z?fdudRjm70ULQ*vS?C6Se73HPJQ0&a!>DyaOq>65pjV^*Eo_o8LrF1@{9S)i<1c{V zroDGc8ALdnPR~)aRiiPYhTPzOzb&X9@_Keua5!_|9fLYE%)vFPbF|dB8wTss;(Vb? zp3iGxu9W$e@|r+mb2Y5YR6b2TJY2=dmPK01$j~OUDsQvA765sI+tCu!J)!-`N*lpq zTO_B(zYual!X|WkveVocz5vB;xV;zIH%AdEVbnlKP$_i`%hl-`YHJfKD0|Y4U_dQm`uL|%?sw6Jh!Hl0DGM1RY3 z^C-}z&H$G&OHUoMyl$ons2qO?Vef4#e@>5CUtuxY;p~GNVofzk;X8x`U$1_P_2kxH zUOsEG%~fWmU_#D9I{oY5PCEAr{62He$U|3$r7MYdNH?!1saVe$g1QAM)>@fDcw1VF zENvN0t{U~N9|B@7%jRt1r!w)JPuo9({9kfNktj>n(ZA0Um%_khJkQWI-pdT0o}o~c zf|)D!r^5qS*B8FUbA%D|qWi{tp|x($r&CN*!|&J0Qd-Ya^~4ye^2V^HP>cyp6bOur z>6FGq%XG@gxXW~|X3lP|D?V`yP{TMZREi3aS~lEv-uXlf)$HupAU9y=!j~CK8*_tK zrpE(EKlEMug_B0+$Oh)fMtr*iNFjy}`9NEKK`Zl>9Am>1GgTEqAp!vyaA)xcRC4#c z;AL`4mXD7%;iLeIGxP-+#t7ncd;Wv@NYgOHafS9+eIBAFEk2zr=-&M~>+IvwYJpbn zjupCeNzc!v)u2Q)ovIpYx`FriR+gi!5ixmO#TuPNIbM;f3%-hq z&iW&NTk9HG1y7t;*rQ9|4J7|Fz(2q1;`qE6yJFZ*?aiC2aukAVAy7EhLS{t}&a8UjwU3>M=(+$Tf8kr937<;GB zQW^}5B4bC6cAn(7shyoCuJkd;VfLG-d0bk8`F`8Uymjr&9pNv1xE|!-)Nc<#(#giJ zPJ;0`+U(}NW>v}!C8-8Mt?P@}gIS_ck59{XqNY-ONVLDH>VV_<+ep!RW+3cF>0^s->*0FR%ML&Bhv3vW@A?Visiu;v z-Z|ktL&DcfSFcxZ` zZ?$bte3B=#5`?*0jH7-*`SjOUv-i_CFCC|`cnj{z)reb@>FEe5_iqO#v9_DZr8d9% zb>~BgdY940LBv7tko@WpHmp3cslB&7L%UO&7hTmNNcKTFJKS(Rz(=t?pZR3~>Rt+D zr08UQb_y%dRWZZZxqmzKZmboxpnEx1kv1hm}qh@xp*_OR)5kK~zn{5bz9gvTE;W+#~ zVX7cei#3{F104|IsN5cpDz73(k|XB3%cVlndFu{*(=q@1X=BjI(Lp!xJhf+`D-3ur zEl&=w1gXT$Kn%gnFgU$$VTKJF21ZM}EDS{XWi%s+hv63v)!eDUK|Ytgg=DOW$W-%m zlh>us6!f5nPt)QXR?R}O?wX%9c^p5;RVfTJ>Wb1=YDOxznbHk*^M(lEE4@$80HdKd zzT2k%US<^^R1H+6#^V;?S4sugMrStr=EKN#4 zLo&0@bXRVs-W9tqvEu7f&~^+D1HhKaO#vj=43F@f^aOU@bJQxRD=!-e7!#_>&h(lV z)dBSsI`9O54)t4xspo66#s{kBc z9sKnE$qePOLG~}jVLKiI$}jp_HA<92z@h6neEZCe>f0DE7j_6PbUi43H%~s-FNMW0 zr6^NDu))eGsm#sy8*F#->xRBnp+z$IDC_zJtG#Z2`6!B30kh@h;}~R-!0KpD$;@Gn z=_v=(g^7j+EOd3+7v-e4sr%I4Ya&sfg%4BTRo$G6rO({nV56~ zIa7NqAAAj%dq@S77+CXoOqbr0W!HbjM{{eY{M>moUlCuR)-#7^rrPz6GTDR@qefcv zNr~lv9KcG$IK=B6PTNd#?RhJY(k$(LCKudABTw3v)d!1tKCP6w0o3lAzl%yC)=Aa9 zx_uB+A^Z#$7hR5sYVU+(iqZP0PN$1Ui1#!vVN06BJKE=Cs$c6h?aX_>V_EXk?9C^G z*a@m+vXOFFJrpG3SAqsA&>Lh2OrL1(wV_tK9hZhV^+D3C)_SiA8!nvMtC?@_wFLxz za;zUVDgtszV{4Y<1uwzVXeDoVFsfbXBu=}atBl?Y6Y~>aau-q_p|G6YTo zc^H@=C-(<|OTA$0a_OttWd_;3T+r{>%w?9|iYG=UVf?P~OH|{)0ReE(Ht^Y1*9ecx zMBXsA&fT75V?0TdTY@?fmpYOj$Q@@mpIPZnF8vMs(Bo z2NRD@es=`b#b^sfO`p5g{J>Dn+ukHPw)gJ`_*jEkS2xasurkPALNE#2n%l*IBF>mN zI2lM%s!xvXqG%V6ij3art&#Fl9X!AtJ3r3va#Q(ufI~{Bs`;e9n3_hNOX`~WnCW*Vg||uL%>hH)`b(7mz1jVK{GVM<+-eBSaPn&W zja#ficA0xB$sdNP)%12G`ot79om+Zv8tX&|hUu=cwP{ZiqoT@c05UB|A)Sj&(5O)0 zIyLnveIz?&t-JEXx1kVP`u@;PIW{g{Z-n;4!#a8qwdBPwPp=e;RX)t&XrC5835Pz7mnKJLJNkZy<6VgAz{{KMm?++%&T$2hGq(Z{aE zJQ6!e$Dbr?Y#9J%^?#qO@aB)(7r2josQx3@)23`pWy=dkK z>C8d=yVAcJ)BpY(=FiPvHW9IAvVqe$5h_YUn%N~WRJ;sMK?P4;l(ujPt^WID%1_Tx zJm6PR)9EM<_bb(9^+!4JPa^+n(}mwt12(;zIti1vt*|n@2=7IKrIVCU+kYAKUz`0j zh@$XVc3u7xjOO>P=(ht29HfV-Bgvg~kF3TvqrKckvd^#p8)Y)GL3rpw`P*5)de}Z6 zh2MQM6LFE!rubtre|0ySrNx@_JLwYfhm;ZoA^_P&RuQhdWE8{u&D<*=2za99ji8VjsS@Aka<0+uvY1`2VNczsjWPLWRyVt$x`vvud zU3@>!Ps0XG<)=5uz=b{!4??B-Q_pW<1%$q{9qQ-hu+mokhQNusD@eQZk)f9uZg&H1=UyiovoONYt^4&p={4YmomWuB1p7!UF$j07x4dj%zl z_=Nsu+JAF^bxDEd^pZ`xz<>75nx*BN1J=u_95qpsZl?jWSeQb$cJfk=z>xjVOAqLR zPh83`zC&@>1P=axzVt^$H?D^9;`sY)%pr5>b(|8FR)zF74san$iuhlzA};u*A(*Q@ z*Iyc9Mh0x!vv%Iv!(*Ad#}`Ssd~K6c0dghS*Wca85W+}1$*H?-Tz_fu7SL;)CzTN;&-=zlDtQ+0F)_W7eq$P=Y#F_j$t8WEUhja^@{Nm zi?YeqG^ty?w9b}QYu z)=^u$a8YAp_!CvDs-f7OU1Li5`0H9AtoDBqTfD4g0cisJMDS zj==izoX|T3P5n1#{rAxt@O?*7qKx-Q%Wo1wjr>NZ<&30G)SVulZ z(7XoTD* zI^bSpFJBa!%;wTg+(MK+@?b&s4-PN?pRV^^*;gWm#OeTh&Ly&L~ z8>mLmLPj)UcocxbJI$Vlr4&UCYI`Stu`n8sy*`~;6%>+>fse1w@9|i_`W*$CcsTL5 z6B20gI-KY8!5DE5F^~CJT7j}~`o^G-0TgTLDI2mC^!5TSc+B zLFj12^5E!b6oz@N!Ror$z$EZo|GU}DgPP2jqwMt4N3?zqiJr8?QV->mTc8tgDgz{+aL07r+yoB%`Z6 zcplU4MD+<=DVhWNfM8mqK!E>(UAO7Y%?`MCXJPx#*g!Ms4_II*0jq+Jd?Pj3 zJ#BO=x(0Bg3@nk8ark`i5&g6~b5j?S+uGGHViMqPOdfQtKgE6V`ueP&hQz6#l9#12 zRyy-tUkn3AI|HVIX7gy^g>i3`N)=Ykrx5` zaLExs%y_`(o77QnQX6Vj*sKUWZUz1A_1+ECx67USy-5P9 z*+T-kN{8Lai5%9xgLTdd`JIVDW3t;6Iu{SC?Tz*0(oK%tX!l;F(%!D5&?9#DLZS^36dJF`{%NQH zThsbvfwpvn*SA0>e|srUOaYbv9+odrrv2FzH;>S~QNxMBZ8Z*Sh=GH)L;0LV_TYXM zEHeKeX>T1?)%Lx8D+VAc-6bvEC@n~LcSv`4iy+&NN`X}VBPNp1#t?uN#~*FPAXj(jR~)7Q-bjoe+H z51$Pld-npH)pElepSHPx%FAU;@P5?&STK?$y`?Q;ELBc@6_mSS)dAj zT^0G}32^55w!_F3pb|oGWn;*TkNkVM%TRJ&498Ab*0ix;{WehHupWA+AS_rMgm6B>q{JepvjK$zDLi|wHi z6P>pi`xaX^Ce!jzus|c3m&4g!DnQ(dG}hS9Pb%rV&K%B#MyQM?Qz-B?il&yFeBZwz z2xF7Dd#jJwE$dq7!xm1c7u29y*Qid>zX7UA9muXKGxhtLuwD3>d~sype1Z(jBa_^JB`w#i}%zfh}DX;^pfw3Kw{D z2rXKFROe$zGLm1`@9C0qw!0dE;A9W|F>hxz-=HIuxln7vuKoyT`->AW$U^rbdpzs> zPPGq0+K2ca=s zG_i8}^cmTK1QiwegVEUZV#ikXnX{2|J(3ssL~Q3-%)7Y=HR|YCSRVedqg%r@0B{eC zE(Mcf?U^J2j00FKRp8YoLp;4jn(|4YiZ#3*62wAK^hj(feN7Ev`9IO0vBx)pJJms= z;&f+5{`2!t;{~{E-4WSd-I0`fu9Aa80{E*ozQ`OB^D=p2h^VxhO_58yNtHe+;bpbo z>Cb54nA`}8k$*U8WU_U>eMDBDOT}G&s({vZR1WuMd^@`5OYE}A&(%Ze-3_u$Mo`Ch zrc|pvv92H|wo!+P^R=fI0lGImz`@07)ywYDui>Wa8viRCulW8@@ICHdk6SSPkSr4X zN+aDhR=QLCBfXYij-wxr&gd>oJ?#u{Bra^^CMDAVbnjoqlSXnzhnv<73Rj0{Vqm#tyn(oZ z!SxKYBc-7i4S6^GwB)iCDM(3=G6KdNimBa{DTCgHSZ>hr-c)ca3K15PRoNZcm(6BH z5=$q~t_n{POM^m4z#PGWM3HP^2c@R{$rzNIbl+;8%7m2eA^Bq=Oi$Lk-NBa_{)pt) zgf$l@95Hm+W`N@e^QH5W$C4G7vh-rrqG~{_OWzN;@fXU0Vh11+=cf&kt5Ly`j4B{* z=B=1$Z38W~7sW^Sr>Ek+jAHuAQXPZd{%A(fcdQcqjjtfHYbzxMz()D?l}&=eE}`$DlInb+l5 zbW877CypowGO@5ajb^7Vo%<5H?P+zoJ^|r%pGdMY21J^4qPHN&7qcixmT<2D*GC?y zK)G3xU7H>W#S$eH~KO;MDfCelL9CRN4TM&S6NwyvX^VpggkKn`8WOn6bi zerHIxJNrT!XK+eP%8$ZHsn{l&jgYIDwu{G>ZBO=w)Xj+}tKoSJ|w(SSY4&>}@{*doRy6E?(F6ph?r=?#S z$jZt-KVN;Umm{C7e@$@MsWoS1I>_i+2$ zc9THwnu+UoTX?`6VWr&}$L$+)1$~|SY-)A%=FV^~C(ttxtL1&T0i#a9^viZ7saZqf z0MeAKvKfa812sCGQ>6$t*2dosrLH=hjWA3=mcv}q`;smOcqe10?ji>S>RAIl8AJ4| z2YkQZ&F_o18O}a%Z0$@I>3p!HWUgD(K)0Wg_LAoq#cNo?KExs1?YruXNu+o#*|K12TZGy5yoxA3<5 zO+dR4&-s(Fqot!{Y(<}w1QshL$~@cRO4IX>hjk%rsUoEg7hT+M(F9}Yh?h?UYGh4M zE~Psh^?;c&+P-d*J($L^$%Q5ya}e$~V7AYM1g41um89DYR7voHkJMa!j86)GSn_FM znK+sea4Oom|{68)Zq`e`TW(-1l44l)@o@#h_+&t7Iybo3Al zA>kdCB$-~i_o(yIXVo;Mw`F)mBGQbs0!O1`E{EVa8ePm@JuY(rm*cQ^6f#NzKIqR5%)m5%FyCTy7QO!(^=^5=gR^zIt}ey;0OSl?dD*J_`( zadDNu(H=tu|LYnBhxzdRc9_2+uMH?6J4t&XZ=ATTkH^v{{v3HO_Y@r0^z6A4CFpm! z`3LK45e4fPM)6OaTKk1FEYd!Fzi%nC9ab;LYlB8?7>~%@TM`5z4oFh+=rxfreh(FY zyrKCgFiBLGU`a<2Klu-SP}Dv8`z#3X_I`zJp`LyZhre@PV2T6U$LX2S)Wa$1{WEq& zz;Q&sQw4M)dy8#}^rk?Sx9vHq+4=gHOzZb1dt)$YaG&M{Ijo&B1j5mP>QJqqn_7+W zG||X~>_Xx;VHH#G&jZNU4{RO&d;ngej~#ujN3CdkN4v`ith+!zAo@RHlkW=-@Edg{ zn(u)r0}Dj12e+vwgKV3$Fi#GVeTsV~EI@4;y$wsfiv$B@W`dgi2~ZAayDigpv_{X= zCQ4S`Z6WQ5XNj4`FM+f^Vm}( zX4Vt>)~ZgZU0_Ij!hT>P`saRj@cAzphkHsbu5aC?01UF$Q5Y6t>t`>1#aQ2;hb`xP z?tyCa)Rpv%+2t{RcjG-M%Xq@O_8PlM z(e2N6^rI95?m=PSUqPWHt`d1wH9@mZS9`sKO%YHO9IyyH)Zn>DEmcMDR$6LF019&s zs<#ba@B!U+7+9^Zv#48#(Hckdcoo;;5x3!>K<`-}_d zepUX8e7aD%!Q)UMsqSAn;cA-N4Plk_&a=t-DIw9V@!4--q>(tX&E5=mqROM5s?;4x zT;Rw|?3#xDm(kz<*w0HuJ{1u8R#y;M3b+6w3s!CAQb5>Z{a0bjpTBW#@(VgNEBkMB z=#OglT0J>U=tlRQ=zWIP#ZW150FQ88cPLMQPdH{6@e$jGo1~6Nl0oDw2~7zIX$OnV zujt%rc`3_BU>c->j^xA+E(wa!h{huVYIj)a`@@l=Az=l;;(;&CFWis}9z0-^8Yr14 z(UJ|$A}|$CciXv1PV}kr6I0J6(qtVh(L8K&P`3Zj)G|GD92JA=Gb%NGHXMEx9{m0j7z#SoGyn?Q9r#L9XBA*_O( zoq9WaP|E4l)56u>%u!Pmpud95)(hUufSDkA2a)Lpsm0QPJ4bPlqZUDx>hLF1-F0~CSt>Xjkd^OCfyRE{ znOf2JQ@)-b=`zFrAzk(@WAnK&7Bd1@Yt=EM!eVVvZTmdK zEYITKQDvnGJ8Z3=Gi+a;WcsRP%2X^H*8b*{-Y>%m!Ib_G5}Mv#o9A5;(9jnS_kI%+nkbD_Oq8s>QzNj~ z?pyq<*lvsy?xTSC&~o+o$y@VfD($|6oS}6r>1;c#eqbTvT%TcnEwe1#x|wkrOyTrY zsoGvXu{)M4BZ6E~Mb%`s1o9A0ZA4r!d6&`XkSvWa_!&@Dgr|4&L~P!lQ=!b-%4|R~ z6yMC%H`iAnamxokgju#cZIi{mJFy=;mG_c#;fT$6y3W;x*_yk}FM1zHLtv`yp7elo@LA~xB! z7!C+mdU*^+O7K3p_2)XtzDt<48(6u?f^CMACrha-LafaYR=`a^WVgEHYUcjL0zBEE zK!0DtKcZjTXox68My)+u1%Xl`^RB3g<8pP%fmGbAf=O(z#u?g{7ylGQ;46S>bBjMj zjAT50@PnL633vUUJ#D1Ley8lj$bmuT%Bgky?lnPyJU>1t3m_`ng8QjeukC zqEYyutouMwktvz-X?*}iM6s5y3kOgwaldt3EeA`N8!&>8%#V1nifP|ZQ?k|WAsz9zQvBz2G6jREuc=-#O{1NLlO5(l}55sk& zP$_xs(f)~z`)h7H@Azpy5~BQtq@P*pdt8X$C52(Hc~S`gMPcSQMIqWv+du@gM$f>& z1idq(bDB>XS(3zHg)Fh|P0?;ML9MAh1C&k(oz?0DPq(~g4wbo z6EFOuJj~hd9Pf#ZzFK>q%l?*$LD=5eesqr7q}0fHnBN!Uz%%lfGht$vd^by3(qyVn zKqQw^F=-smrgvDoxek0~S!MKF&I3)lZub6CJ_~j={S%ejBu@L)N5RgTeID%2Bd`hg z9IlQ1&4=ZpVwiy;R(TpkyBIEBGoSNl6>d<$+jX?tS(P^4JFUJUU1w$cO@(+UP&|68vUT{O_4t}b*uK>t zkQfK{<2@tBPlB>P-Zk`Ysk0?UG0vIQzO+QO?3zdkG_XlvJK~C>y&gYO^P6I3D&^sa zC!?o-lyR|Cm>+8@k?s>uZNx?*o#z3V6JFUEn~=%TDP|7$rVVy6=uNZmU`oLTPL=O^ z$y;P>T}J=2#Lx$@Tf7^!J{qOxyOc{^IqCB(0zdN?jBWqUghvakh!8gvrb{v~1f~%V zsJXb$V~pTj_2Zw)4iX85#w0l_*Wj7Vj!}zo#|V%t?;#g_)xdRTvxUiU??1krycAAR zoT_j}R=m4~56$r74Hp#9K+y zV|ke1%YZ+i^Xrq}p)+5+px&8uS!SfHOx=mL9+_&f+}Ss^;pacqhT|T&UQNF;XQOY- zIRI{Amt%LTOq`ZX9x1DE=K`Wspf%S-`vedi78LN4-%_g9e*(Ho;K?6qMj=o=%vY=! zqvyDjaG+S0MCf%=%$h-p1hyLSu>k~XF^LfYse9zSflviBN&L(u1I)t5_PMVW zV2^;75b|T%;g_+%p4HYB-Yr%u!z$>$#y)j7DxOYbEi{6A(zoRQ-V<732lw(CY!_UKtEGyagbV7wtgUk z@i^xWJ3&Y_(NN`EvqP@}*cw2~4p?`Ph^6bL_26L1aCgcL#_mra!}&w+&U{}Pm(8+X zsvSW4e&vD8I@_D^l~9#295p#dOcxA)pC5B_Al)b@y;YWgByzSrqgTZj)K2smZuzkK zr<&BYMd+tKq$%tF(uaK8%WRDwV`bwPY--IgmF=W$YS&~bgaZh?220bNz_g3hR9tg8 z2_;Xdo|D&ACRafcDz&gaMwC~nYaBmZU`!Y2kK?L(b}H~zDf#h$Jwz0pqIWuM)I7UD zEP_Qyu#9@rM#8U^q%Mf2yvHH5&E5Q>1Bb+4%VBO(f^y!uwkHECB<--LRB8BWe_Jg z8W0+;Q!kNoLgo!(vHGw8i?jIZb;&7FueiRG&pHN`)%B8nx_yS)<~+*9^jMZc79g+S z^|V$l%Rk4WDyn2lyy!)Eeq%|O0p26MxGqc}$Nny(C;DW3+ayDyDrbNMwk232INP#6 zkv$jiU&t65jO2M9A@}KBeG)K@AnX?$B}+tRygOyy%3AKAH<~>JcfN6YkwbD&lb16? z^2tJqd57dnJpq3to5iq+Vi~#SVOPormB7*N?sLwM^o>32kTkUIesw(ua0Wjv1fC~S z1MFgZPGO%8mnju09uAd6J=-gp$Jc;KS;JYK>Ci!MS1%L*=m6Q#)K2(Vje70(i<(k z{syi@dt3ZUJCm&|My5!5?Y^)l9ADNjqZ#Q5#p{6{+kVBW(S~Vk6)SwW+LWr*2pSz; zuR*zwH5ADOSMvY`CmaU%1NPpV35EAcmxDR|E$K<~RYFhjc$VD#Lw}-`9vG??k!TIf znG5@jPm}(%>i2-CJiin+UY+Q+_3VditbgsF;x1rYugiOkk6BPEEOE?(%&k>`irqKc zJ=quGq6mQS;Nj}Af_4O<-g6lC(`!!;k1RfkBYKY3%mGb*(f0_5F1Q~i;i-+xs75~s zBG6x>GE@Y7L8ctv3>ZA4J40?gfU3DaQa2t|{E^P1fuVXVs?#bmQEFwGDPKH9R=Qx| za8PL9%bCjh=a%>e7I`JSYu1VhZAi z^lp(5$XuJTyX+=>+fHA(!u}#lQM6$Q&&RRu!qAbN@FD+~oAmu1wIJV1UAjZ(nSR{- zS4d!PIai7)^aT3fy%=2C6T}qvQm_BO13xISYh?7jcI;xq+~oV+feiSV+99%qP*GAm z$p%t*TwX9id;=&q!2a_+aS;HYjs#le2RCb!B)U(>$IAbY>G+?c=>FmrHK2})K}lpf z6Z(cg_7fpl$N^2%q9K}MaVT9iZrYC^{`QD%{kJstuYZMudiX5WdxhU3@%v{hhO>oL z*U9ts(WY`VqO{dI;r#kJNf)o~cV*-sr-<+O$dv12Qx+Z*#&_u`-~MB`Pvg<^I2Mn7 z?%V%x5%l0*y!ZgxTk!tHT_eNq^@=sx850W82+XRZqL0sh%XR+D6@0q@{L!BWblM{l zzUvk+U{IuHRL8my;IpH@>uUCSR&m9(|1rD7q*zMZrK$4U@creljkNedKo@u0G$C(z zqJXvwifoB=o?RuoVv{>#Pqx9#?vH+`G>iKKZZCIIppl5BFaVz=hpsN)adPBh3?TA6 zK&ghjXCGcJk9)YRx;722&N;cneBuxG}&BIJBuOm3-^-2lTV7YuXRT- z-kaMS8zSxgcpp;P0^KQXVZ~ID{7X|StLBC)>qTGc=9Kd+=~N1!^sAQ|-qVR?^w#lC z--MYGhyE}{bDEz}RzJ-)3M@E!{gbixV{nrGR2~8D7vEu26%=?F3 z9mq%4pmJ8bpU8p~P9noym~gWYK_|f}n85P6 zl~NRr4OkE$?@-Ob?+57zpO1jHISz$06Zpe050A-zxO~ur!6|>d`(|0?M>4>y`EMG{ zp99?cQ{w?V?|$#}hV9&*+v_XkQe)|`tp|7V5Fixy_q;EY00)daFISRcp$$31NKTs{ z1VXWxM1eD1646VnNl$i$g7$S-c*`<98Gq9qLKpf0S^;r@7Iie6UZ6}@&h{-(q|VRH^K7YTa?X=PfkCli zn*^@>F-6Yn@mcK44SG9!d(KfjazR4My0LU{LLm%)MV!houNWTgSgh6{wMWNQ2S3Gv znekTAoNH5vIhzar2lO+pDz!U4b$%g?#=wh4&{Lpt1kYeHk+!3x6q1tf^O!=eh{}5V z8XWMQ27*KH$~#(aj=y*^yFUbI3yl9LmNM6>@Za?G(zBmh_7_6OVcI$=mw6> z+4B0#5*|TKX7lyu(icaOr+jM1=}u+SC_rXR9p?i?{Ah4oiOu`vFbjj^Z_;jqV95d? zjSFOREq>aE0bLL62(Qw$**{W;Z&3h?Fj%v`FmJ%|Y=%-oy)~GdYyrGM@u!~f|fR?ZG3b}$|-L*(3>U`mHbeM2MUk& zto5>Iys6~#yO{O;#|{eMn3a{3!%Z*A(MVm(zj=+b{kb8qzI%-e)`T$F5&1)^7LQTo zb)IyW0L8Df9?#H3Y~kt$u5?NJ;P`<~_1PR5?ct0`aV~Ce9M-SCv{yNa3cbZtFeAK1^>N>$c}qJJxb4nMQ^TYJ|d>S1ctsnK|wIHzEJes6z$mp8HMMD6Nh!t&EjdBF#9@6)|}512T#ERNSYTLa{I!JvHxf z$e&oTo^mA` zea@K7K*rF?^utGZsTMOcGcfi zjCd*df!f_JU$XmbqlwEXu`@QSj~Fb-c>P$#0FluVIL}+VkH_f6&+TNSi0vu5uRpPH zc~{T$!gPL~P1=PX8KhbV1BKMf6Bf8cVEnHzvu^rN8}WFSmr=dc<&e>xp?GjO zdn?KtCj9tKbT+G1NobH1*Y(w!iIct&|I@lj)&e5k`)fB+YHv8B1R2y@^SEG zyLhe~=$WJRaW);l8_RmBv+i_OH7JqG&v@n(IpTUBb8|SS&}cMYF+1?;bk7@idA8O@ z6tGU&ZjB139rNW6q7ZTWAYgU;uo<&b0+Q@3rR*5C2A4ZmByPO5>90=Ovp|EJ7gNa-hM~;v zB%m;;lTz)`arDsW&`C8bHudvVtGD-bZyLz?fc?OGK()c1CRI2{+a1}LY*UJ&pU!9>;`yD z^h389EjET*&BecFz7D`()fDnK3tJcGzpAy_GsdjfL+L2Zkxs>$B@v%o_px?Nlh`Ko z{z4EFJbULK`-Vz;+qC;DF!7{khWMzJHfsdv^wo6%-+Z&=>b>Mx9frzXucmJe`^pDdHQ}o$)2?85!-z|gMX5fa&KoEDOibZZ!Qes+)&?a65y|BWy0qF& zcHiRA9mvb~QOfr}jZe6))KHu)%!*08I7Iwd>!ZnPcZtG=`4;R5+0hN#MIi@QGB#Ot z?n{lc{)v7$`OurYa#)~u8wb#(t=PG_@U>)4oL0}J<{pSOOAG|3;&fsKluf~7Tc>|F zMsuS*YdKwy?$d`^tEUDS2clrBN1atXEPy%IEV+!u_bM2kiq!lW#ryR#fY&l;r!uyx zr0{Z!{g7CDq5o5U{>K6QAzYE@JML!!Wl7&`#09F1PTC)>Dg%Us3LcBtJ)uxgLshJ{ z{~Q38FH@d@!~>X#5(BCb-1tet3kLcUm>)Il?u46Gar)Xn4AmwrrUQe&=G6=RL}Fzk z7(RG<937eyu=*44afwbUE{*29C|m>SKK^{GY!O!Us;m+?lf8w%JBj(qmi}AZDiPi% zgG&W8VfGP@9V1mMOBWcL67g+-LUK-xBquhR`@#1HGXBo@%z$z=uNBUP5$mtgcb`F4 zf998L+d8T~(pjqSn>pwelJ9;`CVv@EAZ#>DM7|o#`d&I`NxZpjzVr)TIW(*QG&XD2 z(9UhC6DdhfS>;N1y}vKA+i_JLf7n`b^A&xdHps}TOS3zP&HI?V|4n8N0SRtS9oH)? zCKj5>^V-HIx^oq@04$VDIhe>AQf|b7_KF= zymhQRD7u>HsLl?U!Fhu!beUGbY@UE;^uJ6{*N|BbV>C1BkCj&%LseZ3j=Uh^_`7l| zdW~CqHTuAT9}sLs#eV6J`bPi03%vf+b=`lrE(&#bx6>*ulh!7Zt{Ui%^wxzf2SvNh zllIKpSfkqo*k#F@lFRFTeMwn{GKPp<@~?W&gX~VP1&^~mdCg zYh*Q|_48s0RxZaD@&;W} z#2VYOY)}a#HRqEj?&2DB`IkaU&z=Q{UZ9VK72RK=Lsu>BMqL|HAq1NM#M7%mIz-IKo!Xd62`@5jGl{ zx}@W;hGH{b-gOzAiR;R70UIjFdj zVE~5JIA_F7!82<44g8ZW^!L|z1%maeY|26aya;39dV^_g7f@?dmM!lP*?JT*4|X24 zoYd7lmf3+GR?jMKuDX}79{r1i^{@AkGUQ{oZ(Vi0Lv3yC(osJTp4Fz*ILHFM4)8%x zrxH%l>aYXhFzqXtDu<_o(~SA7YgJXSfJD^(sg2!*(rI@HxJ%t38rTt%(*?uXb8Em7 zpB(aCpbZ${e=4$f^v!V12LD z)^w@G<(sQMW?K&YX zxT)du;K&!Ri!FoY$v&pM2$iLKC887(4Ih*z)Y1LSBQ#TU75_CUElugCFOmRj(cye$ zcZg605BJ81jKAhM8k597x~^n?wRbJH>9oF&RV;J#POL$k{zaZp_94poff!I+88Wjk z&Xy1cK1UMJ{Q+mmFtNxx&QW3QJ~eiUtP~56vLE31SG@SgJ^=n$Wv+JM%WKd3fz*iM zTBbL&AR$&%%^#S*q~$LT4Gr!7*;A1L;r_%S8cmX(9!LZXjQ)tmo^GAsH)B(Bdkg+7 zYGdMLx~z_Yd(IVagy&}o@}EUB-obQs+sSwkM5E%zF^5E2#O398^u+LNuPq}or>-Mu zBE|Dco2nc=KR1KP!Cn;SUwR(ys) z=+kU?pe|FJEY0HKNcX`7Ez;g7k9HqxNafK5qYfYGpBk<|p2VO3V)|}Auvrq*ZxXu( z(;OV9(s<&{JOpr}Nx|PAOdp)7(yK?}aX3T!2)J{^mf|O?;DI#@A)JVU-Ou}?J(!-S z*JZDh%vu{f)P(a^36abr(j?)F3A2tufgN1vJJ>y&4l+WdTGnMPWdE6mG^M_6s}JlZ zS{92!doUKgsrgi>gIXpzw2N_l)ZS@R2ff{XMd@j4_%b9~n9Q2Fvzr2~_K~JGvVZTx z!xHaSHKJ(77I=-II`voH^{v?W4R#?nx4v%tbG_lAQ(_c7}yLX2JdpWB8{ClhqGbLnU~Y2#!9DQq8WM5 z7EjTR)!rjrn|xVIZl+oL{^6fe(X_%D~6BX8hVB?@DNJC`nO7 z5<;Rwldxj%Eh>K*X8-#R{Ps;VsUSl<2kj5HIzRRasu$P=RI4e~{|c%Bv%23i7R4WC z^&KBXV%M|UQ_js{JyV=@FpmHfFZ4GB(T6x~ze!mg|53_{!DcUZMg0E{Rx1*3$z{qo zRn^oi%sDY1v_`3kI}YcFBU>4~X(@trGd%QHzs9CiQc?={#0R+JHM_k4#?YN0iE`w* zA0OKT1a3@roX+At?SKc-z>Ni4&A&kXE$_Lg1EfLWso34-1ZikR8>gQ;xBVq6`j_uo z^mKG%P!5v85gJc?s zUBJ!`wBBw7?ChhG!q zilf^x(i1c*1BH8;H4R(`g9a7gWR^0&fsXE&cL^#G=JN5e{oO!`{w-1j}`a7bzX!3 zMKfdWT{*$D+V<9DfysZoBYUj+yeSzEJWOs3eE}ABeRNB?J5FEPK%vhc9u`WH_~1D8 zA*Isea99clBd~azz z`2G0*sZf2JcY2oW69LqAsP*9@cl9NCR&nO(Y%iRUkYLZc$o?y<4k8prLo~8Sc<=Ed zNKG*Ghd|ZK>B^>A+B?rgHY=wc{kD)AUtnzhEzsRN%rQ3(4c(WDAg4gCSjucFw`F+& z$)!T1E<9-_P0b=r9QSCGo?Y!ILRfdEBo(yRGx5;A>u!^tV^^tqf!!_-u0mB@yr?%E zB@Wu<=7h*u<>Zn6XqKl00O=$Euhh-9;B!fqrl#H8av2Ul_Az3I0*XGP7m0i8bA91> zuV;OUIT?7kLYliiCU{Qrfh>kW-<$J^?>si5e68ig8Bn_GBBGaq2o zal!p=WSjh0Kf)F}A-gn6q1k`x%emH7nB1C|7?NUlxTV$OUJR;VZe!# z2>&N_4qgad zUrnOi{KYxKYj@%y!WNX$-5`i=pzcWyM1KR$w~CVlX{^~=s6Z6QaLGGQ8N#`jSScoVSd-fAeMY*|+x%g9Tv9b88~{0v z5KmF#M|7Gf~zG0eW`Vs3xHf_}zZe$mSW=>e+5Uk=P2P zUk27)K>Mf8fE}HT2Wdrd^rAD z65zG4#2l1vxJJ;JUJtUYxrNWjWvZ^U^pV*Jgb4iT3wP`nF+HQ4=)vlh*R6Mq&a24g zN6pKtGj_gm9t;#(co_bv@(-W>KuX5>?o=T<&_UAM+K5U0cpp-CD4E|+QQ?e#=POQP zfE1WE$8@a=_Q+iJXH4e1FaK}gH*avU>jq!la;u`DXxVeGtjOWwkN_wdiCI_|!k-Cs zYjlIgYO*yKD_uXV|5|m|2!qK{3^I*6)mT|61eg06bhz;wc4wu_+yo-=v~W`C=2<6) zW2>|vC`1B#4)1AL1%zfLgyEhM4X1?n#GE=f7Ajv7glaz2AmK;O4hg4m3H5VT=SCl)JkQPNPcVI&n;$8|^0vj||qX^RGs3#^$uWe{( z6KUlIInT)*BI>##^WA|il6XdFq9@qU*10aK*hnjs{hx!&+>exId8ywl*UVKkb%ap$@(2nwr-d&od%x<&S6Zn zR^D0T2;LUXb!UDN8V#0nZclhx*mBBzN*C3sBc#B2ju~pj*1ea0UU}veN}bxeHb=wn zYrp^|waVhZI+mVUFj74~RV_ZPC3{y(y?sRe*6OnAcF$Y zftDNFj%S*&@x5VRnxsBe-#^23dav@t!0MT5U>LF4x#@BYFgZX1h0OBl?~UaeAU8+# zK6)={XIVDAcoBJdzOPm#@~{p7@s16)$y!1Qv1xrk^D;n(b^FFyr1w_fhTg*_VXSNi z=$lDqA(FuM5SGLL-o#6vJI3uUZKi=L7|VbON?<|;O!4}9?Sa@syi55hBt=(ha~!s5 zoSg*7*5IsE)DRLYl4v@ux-xobhRvuX_E-%H$(lG_o03Uf zb2q(%i=>OG?24v3y+tCXinS&G#4zYcin5tbK*8I6JEJ~a1v@xZmNH^&BkO!qjTu|c zzm!4vZbAlhzNn!m)=;cOo6iT0HBAIG71LJ?->iw(CL82@0yTjYSI`bS;>bLNQZSq9 zueBfKOrx>NHTBtKvMhN9mXbQ((aLa0sLkat?5jPW+K|!idL?1kVIWfM$34R#CsRiF zIC4!Ji;KZ00|(sYRs~OO0k*H!4nMNjp5Vl+wciL!n8oxa2e)_@>LFOF%Bxm}&H`m- z&D}a1;oPEGwUW!D!Z~i$fQ^a1{wd0B2kf4(I7V{;*(17+r~{MeFCHdQqdpB?pP=U= z^+It~&XsGALufNa1u}IMr(!?hHLNcOH$6YSO4pmHyk3IFjJ?a;gZ#3+0G``*u*56$ zIjz=eMAfe=vVN{+eF$UH#O^F1erHWDbZz1;jjQQ4nE(U>|7nbS2+>W%9y2%m7JQY7 zAk$!b%A_I8MERxjW((u0_Z(h^=y^l4W2He9QDWKN)$OCpagJU}>kaZ3y|30lAm>x> zJksl~2NRlxsmxausWX+b`8m z4JYDUXa$5C0ykV+PaRN^wSgZ+LRRVA0lwb<$JUh%k-<)La7TNZr#zTr>^*Z5&3CNj zl2bgA^-coC9<}mW1;ZWh&Qt?$K}lbQ1C z;P8V;)lsISJG~h+$uJpkxTfXZrWN09oOM>JBVJ}uOCUYoMGnJy6DT? z1r(bh8lL*dwM=^Q<)kx2iMq!2IfiPV%Dx>xsyeZ9|nVh3ife^a-r*>i_JA2U?M0^WW6zEg-#g6KU;asv$XKcR;R?~3yl$Z zjwtctSR!WgwJsu{ieAtwTCu&CNA5>i#dy;ZQi)YPK{#&Lw^vW2saKpPKu6eiV=N2F zGxhedo(;)fam6?3*pN;T#Kw;?XOj_Z%8rrYC$`cO^ks{$E24WZaf z3W{4Sv4>J?V;e&4oZ<;m942OtyJF~L^ZVc04_z4DFl|g9FkU@SP_GZ1*)ix*EN!u* z6oeQsWmr}bkWaQ#aYR4c!nZ1_Mbo*ix z8RHD_m9+Z)D_Ql8KN$_}3pl&WT5I*&_#U&lfbKJ$+H&8Z8KEN^XQ$7J8NH6Y$18d< zx4W9P`haOGePD^6m4NmRTsR2mH5x0E!C}7s^`TBp@2W|dD1*s7Mge6B3kEvtXjz5E zD2P{@XY6{dCzkLM_$0Y{1gMv1HoV%vsk6MHS%;<`GM>#sXC`N4E?Op?ow578Q>U9t zb@$NL;g~8``P^#q@c3f3e|$7kZzv<6$Yh2Oy@K2N8ck@#B9q)qGF`-A5nRn-@%i%} zL(daaj6e`HHe&~tSROukp&7#Qr^Z&v zOLCa#9+jil(YVO3sL-lK>++or50F9z%O>M)ckyfc{bOB1Ggbv*Mx^p=A!E~`P~FeY z$kB!G2B8H3T$OIrp0Q|zCErUqe0dz;;uEI*U{%oC(6>aZdy2z9b@I-onhn%*Obow6 zdYy6c=)m_zrN}d`D~@zW>}i zo^o?>=ko*f+VK;IiL8Mo>v<#I%hbx+Nnzw5kq{u~_14)bu&0||`P{C6`fMz!nYenc z9pmLC*2w+ZE42lbXJ6%S4VM_ROqT>r3)L3gHG)p#f%P_cOVp%ZsS=Vsae7x{fL@T~ zjGA?i`X-)?Nt=H|1xd2-MF!i+SGSQK$%S~$l`hfkaaOifHL>`RLnooYJ>%F7HWp#; z0~I7yVAnlCsL0+w?ff#!j!MJ!aHw_bi?5QCf~3XTsOabDFIy37c%W182RtT~t&?$4_N@ z!^3`$NK>5hnKJ-8zxxJBV=)U({hSfyjxVcDU)*4`lJ6v9o(LNp8kvakSkzLTL1a_9 zu;5a9HxeV#XS)Vi^Y_ zdm_r7mdzEK1&SFq{J|-jV8B_iLl266y*&uGA^)z`(%?M0-VnEzow0z_DUeKAJx_L% z+U}^{&p&#%c_>V3Ps3~>92ix_)PnkF)Hk4(V~356S|n#XiMq(o>^y8lln*g+cakGnXrJJ+P6b{v!&eu8o?CG3Tq|q4Di_6(Cp`48Dk`1BFbtq{OARrUqI3vD!+=UNgfMgr`@$#G=Y973+u!%k z-rslZe>m=giTj=_*0t8T&ULPm8b|B$YixBp20W*ThEv8Xoh_NX`WIB=K0JS&@IJrB z?{ZV+BhikJK?9`WpauLD!sM%xNlbWc*}(nO7vU}O4je`)*jzw`enjc%6H=%7;f<{nO?~LyqQ&95sqESF)S(A zzil>nM7L(+6nv9lhq1YaoNS5SRK3pIHkk0^ldm_620d>y?dsTdMD_Tv zqsH65k8`UfM-SJuV;qY`w@>GdZv~&by7$&jpvIf$hA7xO#9tC!tTs!n8+}GMv=?uN z5#7))n>Bg*BvNp`&tmDYP4`UI+B;K9s9!-iWy)4UN85-fr-86#@{Y9)(MDffMkuw| zH5N5m!EGbx6_)okO+oO&!tlilDAh&JC?O3Ck9UgvxVLKX!tw9HbG)Kul%5!Jk0G=t7A1gBg+n0;q!LDc%GMWs%orWce-8$|4QZT7wSllb}n+qBPYe7 zO*p@CpjH9xUU_@`7|-BJIEZ!@PLa@ETrzb_q;Vee6zV~Hso8IGI~dmz0RwRPXxQ@5&1^Wxn0)mUyf;3Y||R`<-P-0B>egQi|6|> zG0WzU!ei7kNBmT&3!h8Pg4X|}uHp-q!1`phk3 zs6EDmm8?`PnO2QWNvt$$O%sY5DWyT~5?s*3Bcjm7VOEKR_pe9F=GaRt*!c{-c3RLS zspFn~Wy+5H2gV83XWE38P5;D z?dW1a3k|_-eUp3Tjjd(Y^FO60L_)tQN3ezgyxfC4bTd`Mi()g|+TsUP9XGKDAB9bcoYx>*#b zR#r}8XxR~ENY1ro5a+uE_i^<>1<7o9X*tz~e|iqbi7C$<9(J77Q6u1Q`{fA`2ca*P z)EUZIkPmHgu#UEUpr=p1B+1}-dh3Km=cSGE-X~L?bTJs{+*NyS`Uxop^VZ(ZBhi9r zXTbt17yM^VF*T&nRL9a~vxHf?LP3k2G>a6E52MIn@u3)DO<^G*c zWi`+*(-JSwyhX@8u7$~N>R?%4fL9Jc{6teH#Jn1xq1am_KFGdvC|(P zGMF|>3i(c2g$S_RreYKj`1f|*KbvR~fIF^MT~nb@@}lK8#B+Ru0Wb@kE>n6?$w z&I_0R=)C!_&EpT>%ust0Y_jq@yFYtq{`)5a+z)-p<;jJMf8FEX-*@!HH+}zlEAQZv z|2+@+lION3U`?;AuLo~FEt39E10`91ErnYt)UeUzHTWQ?I8Z*-7PSxxi081_Jn*HuV# zwrfik^E~1KaS`3`H>$sO5xJ9Liuk_#faqS?u7Td`7nT#ezpq+u8;+?t%`CIzDo^*lg&SuJvC0#ow*xb&8h?NKWSrcJ`D2!?SE`h1jvng#qAPRZhn47U^5Re4Vx zJ{^;4K&FUmbOJwJtojYbmihtX4fl)I>Y0&%3+iQ1$fk$#XxF;RsB)z(cHfAONnBjK z*mLm~@r7Ed+)+KzJ}n;eb20Fe%;tQ3!BnI+=pS(dNHj)qIlQ-WXe0W^?z!?(6k@YOeK1{kimXV*7C}i}W+P;6kim&4D}_xok$Htsg*Akf0x!)((WAd8ZfeLNSNFfYSS? z&zanos-5-*A5e-xp8@^6hdH9(-z5sW-W5043n;PM#*@VW&eMpGpu_MzI`4Nm-n(^N zQ<3woOs0*>tJqEBww|eAWrCnx!j3C-TWT4#yuVQ6I<;SHxcJWf&V&M3>2)NJxgx@1 zVI+F8Asugy<+5dRHX*kj->|OCbkX%{Z&Nn1^&^0CGkPD-4PM+*ETD=97t%Mp6nPXc z^_z{VzW5{`W4-FN%*qnZZ*AfEjgBGZQ2RP!Ly-fk;w$dXEtc~=ph=4HF3 z_YEw8$E+>RkZ6PcmaVdbT$=>pzSH#SC_-U1CmOmK(`Q=?Q>8^W@0_nw1YMP7a8uyX zQ=cW6|x${vq-pu$ym z=QELP-OjYL^{N@|w%e`%^_PX!c6_Ib!|3TqP23W1fT29xSKItSVwGVwaEf6x!{`pa863dW9M1kmJ8glP`a{O(DIdX(ru-sT%93K z=lwU*4OYne=+VBpm8K*oL)u4##!zZ|+%r&~UAEo@fNG^&s9N*0S8t^B<%oQ|*2{5z z2K&f-UINM&xj%~EhPBroMB)YlZ-Ca)0v=O}BtvfbfflVXx~*9K_kaRtYPGDi^Yyj$ z`H|wqiB*Hm4XB|wM*1@8A1A(o6o>-U?*!8vKb#BY6C3n{mf}D>gilDIeJ*iuIDxf-Am*_JL-ToNhz! z&i%vNIvdbY=TMq$)0fAFVeTrCE-ba#tYFs_80Sppx6GM6Ug}3!585}VFD<3vu}Uko zqoa~{S-2ad-3=b@{DvWVDzpWhRAlI(nGC`7V2ZA`gW7{z$^roa@WmNc%y^OO;H&93 z#z>mB^=tZ&GQFV_Q0M7p)`0?}G8}@>yJzEJ8p$x>`^3Ql&%WbeZ;7jf-gtlM@NU$G z*~dRTR>{X8LQ5FzVwgH~B%>nh*$Ow}jQw_X?Vml9Z;47z5As-b?O!|j;LO(6)^J=> z!hLiG$NiKPk6QwtiG;ZjnI<*J_Rp!0nDg&=ial2nwyMAFk-aS0y%beoP|JSp+5lt6 zEuWn=Xic1cTOoFFp0sYDz_1KUGuH|n5}AZkZ2vfUt4VdoQZUG__ zrLnT~c@dzit73rn^MwlJ3|C?-;kXy#6qI_nNSc2JWb~lu<39fU0vbTCcyZcmWARml zN^1-&4}yFkKiDX-+LvEjQ;mxEjk23a*!IvfQ=wls75ZY6hLA7Nt=F zmn8l$Zt@?Fg>WZ4Q{xPJap)c0IJ(H=)@;i-xlqa5Q)+(v@Wi9EZ|rM_^UAlzWgi}2 ztL_!URrsBC+#%oU@C@GDYokJF;kWRG9O`xUrr{%F`DB~poBkbD!Ehgt!l{@EpvmP#b=gy$0 z9rfcE2`lAdXZC9r)v{d|Q*HmCQw>EpEt%OMfFUKzML$QioCKT&T5s}$RH;Z{3o%tJQe zQfx&yGgR(SNK7+MJs`{cJN&>fblS?N2(%mHv2W)adiPaZ!E#<-n3j88ZqcHAzJcE^ zTxPk$h#howhrN{z9yju1YK?;pd#?3nlY#;Wf!&COSX*u7F3IN$8)5WcanHsH4(R|l z4&k$2dv=9Q-C%XH11E1Y4!WUh(q-*MR10}0?xFYTSs#e59UTVLlQLY{nD={^P=q)` zAdQ4$JCU20TaW zdCXf|fIkPg;;@b_&a>v@0W}Zk=5J!uUE1P8H(IbxQ@0cednXq$HT0amzm0lxK)C&@qp<Yh{`>10AM{$a?(>_!xhuOs#HQ_Qk^HUi_0~Qa zeZ;c152I2?(HP(UHaVxk!`8TbD|DxiWzSWF@8pQXV1a8{RIXGva0^GK9w|MdSULW? ztz347i z)j2!Z1&ZY+xiC==^+V%0H5(@<8F&1@KT}Ax^H^)zSR7f{1N_XAidfKSJEguOv8=G}?EG7%B8^I=Zc7${0C1j)5xTIrb8{~ZdurxFE7`U02oY^MydPeu-1PNy zOEhSlOyaX6=}XNp^Rhv9eDVtQ;&Fk3xwwQ;@?B_2)mQj*$$o`-mn>}@%$4h_&r!X zd{!n#NfW-7M8yjyexLf^!h;_MLX`O8)pe!@r$2`C$G`sdk37$@6ZfLqvnM70_2pk4 zp+5RZz=Qw$kiU$`@1y>INAmyZHTfmkEbQ6wUk?8-oWv)l+Bp`7U7{kW15K-Nx@xH9t)^*>seqiGg%+(SF|P|l6)W2C&G+o( z=5_EI3cWbg!xS*1Rr zIgH0}=SU@V(Z)lU>&(~p3IN0LGOzu5*UJW89 z^t-2pUAe6)Q9$6lODTf#(<-)Xvp=2-wARUd^0(*Y05-?0!Q2_gmyJ0DD|B*!L(*10 zo*%P&ODoPu%qWhuDSW8@N&0TCyg+nQVckA^cs!5tRT1e!N5riC%FS_Kv^o%mAbnKF zzO{Tp9pUUb_wAGXSxFyWbvmt|`2!|LFyz;J1*5lQePI{65IY%}hM32_(dXKJaqMp& z6LEvUz?xLtwD3}wpW>E4yQ3%D?gKAe2Sl znU2q2k57e5_)+-z`H{NkTVq7K)8Ip!u9*|1E%-F2^q}@CZsrs!_GIs>(F#bc)bSTV zExdcDos?)i}xwwZfdoj7_x5} zKd>vyzrkOTBlQ4T>ScI7WWJWB&Bepa1M~=xy%$dXrscZwSlolnKGZ}s)C*!hT=77I z`o5U=;P&bIKx8VY%8?FDof}f+b9Gv!LdN(zG3qUpEb6z@OmolE@A}!^IAI0LExR=z&YG z7{BDxHd3F3dYx}v80lT%Ui&alN1=60--t6_av6@XJft0c>Kags^Qo2xoMN>X7d-Si z4XdoGE{n6JjwRf504;QJf`rSuE5F*FBgcph{C6^Th1u3sF3lr|_u6o8+)vyStABU( z?=y&D<+P08%^W%MXo%=57vA>*=U@8N%_B}L*z{%DHUTge;{D9E;B4u0b2JKRcUCRm zBtr$Eb@#6QooJdGF>&i*f0s`4ua+NXF8S7Ec8$e2e*9b{a5KxJTEr`-BsR_0=sTea zC7VTb$9QQ#nxdt#qfP-szP?aR_#Br!d~5?mFsmtZmvzveApON}3~E7R3~ua~exK5S zk}~UqfH(QhWg=hOv^jSz&5amoHy0B=PEtn+TP1fTvhFY9w*lZMMwo7OF~9R~XtqO)iY3v0p_q zLxu%o5rPiQr<`Fe(j=PY0o@lClOx%*Ih6o$amXi}UQC6G-}`BOR^OLXf1pP{7XB-3 z24>f$c)JOOmwzn$q*_{TeN_`a3wyigKkdIi*=Ma7a8^bF0-Ef_!+?Gcizho|l>*e# z@>0JD`H3_ouITpov>7ggh&PNzCBbE7S;@(TjIYuZ;lrKAf`-jih|#I9oQ7NGV{wiP z6@uyRuhe@AH03-poCSG_X0-+4v$kutoB&VmE8xj(9P;Fc0m=`dr*krG9BB~ju^B}(*WUonnwvr zAas#@voFaBqf;w5BmIDpo}ON$Hk&!V?nzR$&;lJ?!U3MhcXXsn{6GgtDKh9E>@?7{ zVVIM()Tfi24$~OjCS|H1k}L!JwC)KZ(VCc=DyuV+F>_&vdbmn^VBMnGmGQ;GfFX{Kx{0D{uH`&B^olin5qv;LfH;&2Y(3r3TILC zr6e?34MD2>XoVlApj8%mj4@=mm{%nq2fEZe*L?ejF5|N((Hc!7y#*dFIZwP6IzeBH zp{@c6(W$Qqjo4~I^WFk4uT6JZPNP}|obqN_!@Ucr_n$yF;MN8b@#_$M*SJ&*JIzIP zx2b_-B@+`1kqbFELMhKU{yEIfL>=a^yGf6(GI@Vp9c?i?m@BQE+tFw17qKZG57BGO zAmtbl!rbKpQed5VXDB!yH*Qx??=9F{tn_T7(sNK+KA%p=mIaR-X2-VAPPdS%^}+J> z{oL#`CSHqU0Jj?;w_&lv+0qo*wt6pLJUf9XGIx`YZ6#4ENEuztx9+CubD5u1@LC9S z5pq89Dh$&4(*T1pex?8Cf>nKWxL|8IvmB~+p&z|<6H!~ldgDkec?nDN9I9f01whp0J}a%^g5 zv9pQmTfkJP1TlM@i3&TlJ%$Bvtyc*fRogzEp~RO_4kNldYx7(s7HUO?9Yz5AhMvmZ zh3v?%=@M2R?+$6uQGLPl$!axTVckP+C*L;p?$6O32q8((V^*PMf#|uF8nzKi?ei&v zO7cqWy8BPuBWYvCe8a!Lr8)JnWkR3=Apr3g?XSqTN?FZw%rYvAGwVud2=NdeJ)vNX z@~~nX-rGJK@1mxdJH0rG;Sv{^@DM;@`#^VFwL=g|F^(Og@4v)j8cEI$`gl2zoX_zT zF8JO?i=C$fL33RBGwbyn?ALidm+$(QPZok?oLi%+;QPMC9tTOT0kig|yZ$>C?B55k{Iuka z$9e^^1G?D`ubr<_9Bv=aeHQ9QT@#nLb33{B-@n~p*jOH?o;JBxC-$J&tc6peaJSo~ zNK^Mkt$3^GCqvMxbCS)Wnh-}DY}t@8K6#C||MP5Q^wf|&CojH_xsKhW}Buk6hoxu2n&V5*n?qPc?DVT2F#gNOvx$IZ1qXb1yOv1lSZfpTC~ zPOC9lrR}~PLAdT)wE1C1%ZnYXt=Moa_M-QykgvL3A()|wZ3>n0I$w(+UAVAMHP$?R z_tuCg=tF<=(5D+qmKYJ2$)1Qb;y?Puyp3YWWHm_)t$fivRc^>sxi7an9wz-t&Mo(aiMpp=Lh}*?&8o5AIYzS|s^}Cjdx5P4* zrMWE1R5u#NuHJCA;Cy%VzN@iC=EE&x$2S{^`;J1dO?7vPJmO!PXN?!S-Lwam4IN^@NqXfq{jdt~6Bc z9b_?WVa{_^Qh+3m?HuuN+qUdTQ~_m-m#&?qr#B`B&U|K9;+YaoKo@_Jv^oIF2UH@4 zu(b^fx1&hlyylS@%Z9kFNCPxmtvhQee}Snkm>!jSmq_I#VXa!ihEmhFo8zuD8nkEo zFtu;n8HjH*$8+ER0YqJTFdh;U6#I+EDWhL+DAumQ+$;F$+|*qk3}!tt0_LSLzV?uC z$Ci7ibCTPB^NpeDTpHG7Zf!ZtZ!+VR=M1kY+4jzM!jV}O>MMPJm65c&-=(@~{tLs6 zf%S|#c-^$l1Ap&XN)~6hDqVtjRWy1}>?CtpnZ;I0))JSnhn1(=wlVNm*2oa7#ggV* z#co!+A$<00&pag83`(EA)vE=GhmRTYb~q{EgPpoc{tMQ1Q58HqyTRodFtd+w3B7_y zm`ri}s}g{T<)1qWU&tZN3=@eSJxyGMeZ8Ieop*xe`5hR&m~VS0APdYGG@kp#4>`?n zG5Od)p)pJaA=sSgftheFUtl8wrohy{k@=SaV*tAJ_s93TW|1PbxPS+8Uo_A$PRhcl ztR!!G`b!eL%kk>z%8x7q3r~uWul!E7`IuY(;nVUb9AQwm{cxp@hH?B7bV)K04*yx_ zu_ZRPUO$^ugI*A4i>WgvU4ckrqeZL(4B;_xA>ur%$bE^p`qzq%kFV8xgq<|HJ`&@MY*y7d7u@C% zcFo}WD0{K9iVyFlr2DT!B#%*4-~zgpEJBdE+J*8pa@5ULuC{5;nFVoX>%*Gg9FC;{ z#3K&SnA`G(G}7#mL{2Rsw3FE+cKd&p+rifva52F#k985FF$9?)q5^00( zYXpx|!9*kXo0$(Dstvg*U6QZ~W zP6g3TmD8du;rMnp;q64ruqy%2u0P47AGvd@J6~Vi3D=*0yHX`dSQS9*?9nWpQL9t8 zT3ZVxqJb8EYh-HBf_G&x*&(f1prhA%`Pon$$*XbFN^#Lbh&~E-L@nS(IKIO} zm2O5iJjxj<+Q~m7vJ0ld36py~o-nnilc|++$RE57FAcCE6t6?)7p!GWAZIjZA z5gu5C|AToZS&#J36uz;p;-<&`J1;aUQ!>#l9U)37Wb^xP{rimk8;X=~26!&%>d+gz ze~#5pKG`2v7g@kNC!R$r|1{43cGX?flJu7r07#8o1MDG{X3L=BU(Bh0y&-9Ha=HIx zXY5SDlv?9I?%bjQ7V-zNWdd=zRm?B>-GFD;F2?Ydm)c?!?%Iz=IFjBQzFPwvg&0mv z=DSB{IuX}$2M7#6<02|Ep)9W2W3?qNA7MVWTQa$|!g#4?rUj|3S3ZB!ye$R># znV+2m^;D25Cv&z|yy33=&f476_eg0Tmyfni8w+z*Q*U0ifm~z09ty^9tKCAurhTV; z6BTlDl`IMh5m9<|Io*7wbFtW@TXd*)f*zq3dnyMst~wF<1&{QnB~WC%jDdJF`6z z-m$t7|K`zZmT`F`pZ)h|=ePxyyc?J!3y)o~!EPYtu@SJ{am7%RQeT_lmHn|C|ygjeu7ZmaS-vL?EfvP6e4u)(s#>syahF zyozJEv>*5ym20ggkcR{koh#4P$WXR^>dJV7!N=cjxZ3Cp+$Wq)sM}RzraDvfkee4w z;eCtoMQEg#;y4?q>`ok!$qaD_Pc#X9o2%`|_53VZ?EB+#gWh}xNB3XIzrI44zT+1K z_(}?e6h=-<4X*dXxocKFe<_?G^WLvg;n^y?gqq4I;|b{3Hlj0wjcXSbmkJ&o3Z~#} zUc!n@eGSS{>KfUWc)U^ecu4Sv^UmcbrGO9)#Vx%Rt2?L{7kXmiS6c3DcR$ZcK_jU! z%_{O+m~8bg#}zOkk!S9YUwKnYId|i+w)%L;@J(S>D{d-Uj-Fo-I*|Y*8r24R>C9V`tp zQ!BG|KBuo`(O~ItwNK@R=t0+VLWcF$cGfRa0=|~r?k3mv08KI~u8-5|u-qYRJImT} zs4tFHqfwG{vZiZ^#cP8B=pXYyKj>#@PJ47S%!4}d$@Mq;4KdRCB^d%z9e@|tO7BPD zBoGM9^?H~LkGBfECCfO~CQ#3<0JtY`S5u}tcXY(JT`C<%=Fc$)wG2)L#-+OkL!;Va zws8)%5zD??!{(4#yC>iIEA3ZZl>2h~qjuJ*5c{V_`fJfp2iWCLN=g`(1ZvE98Z_Ut zv%h3}l_H{(HmL+5zG+9YZ9<^ID7r^8vW@+E@*X)_4i5Tk2b*Pe;QITDEe+p2mg*K7 z7xME>-;};!hzHVS0@}KVv8WpBbcN-I5A1mzSL1JDW(Thg+c+i?ADibh-?_8vWEVf; z0`nq#@Mw~o+07UqP)co*4)m{jCsU-(T<@+#AXw&F*(#hb0E}>!MI-Iwsa0G}Jce|- zblL?*k=+w__1$dr23kgV=4*$c-aYm?`O4ISHa2apTg{JzT((l?5;ir^1ps(cuX6bn zc$4Z%Nt7x!o$*kBaZ0MED>4Ea={pekC-8PaQ_&44lGfwPM!qV+xDY0`vO<31y|-@P z&IOdi9^C~oFX(G1HW$-XmsiAfKJT8u$rEM~tfZC-XE0>TwfkOaZe7}GdBn5ARWeY! zUHNrAls?v+o{Q&wS6olgehF&#n=<^E`9fC9s(XoCdc5`doZZ6vwZ)*tts20549<|< zx=zm?ZyGq503#~#@%Et^TK&y6di6KH#!VuuF|Hh9>;U+9D|rtr{yu;LZXfGD|9Zah z&4>)`)$=8cB=l9TiyVigTOf~3dQCQ;%5_mEO(P>T{9UJO0LsR002K}SnoZUY1T_;@ zZaMrnIGg&$*s4IVo2A-$jE(F2UHTF7H&kD|D~(AI78BFsY1YA>orX4TYUinv06&v1 zPhI5})>t;7%iXs0?>OXtPyUXKA17qHyaLdGaH>{-N14aax(XRr*glN1IONM<=a};>G#I-h*vWR zTxA*nDM#KB{XDhBSoT3y6k>Zq6H6TL2WTY7A3dZcZbzq{3?SHm20J??KYss7mer5*u}IueY&xCHf(t z1X7Z${f(l1STcZF+YVj}yr-Cbd9xW1Ehf8#*Ucv~)(`2dEr8;az^xplgzpcX%9!K1 zBl|k0vsyjDgVZb3B}c351)oEIe@6Gx%K`+a6PIzVS>W5MC;f@p8&j;L#>Nt)F{&(p zLlC1D=Vu$ey|pDDDreIc$MFU`0rps?sfDPnpPz(C7s=nN>2OFG3Vs}GAvoLDdorg( zoXCESPgm#J4Y`cI?oSWNBklA7+z-;wx}+sk4gj?xwuP>Qhlx7yC|f2=P0Gq zO3z7NtvLzQos(zDiza;MgRV6gl{GCnR8=pIpzExLI^pDR!Anl$HBY`?cj`DH%!nZ8 zLUQdmDXY0HFmH}(^sjUiMYt+|t~(f;H<%W7#!sKu2-9C}>K`|6={eb{*`yqdRov8c z^tJe9-2kRp5b%9BUHh{-#n(Gt-t84+OxUAPS`vLqFkV3V{5-uFE}l9jtYuZybUbx^ z)xx_w>ve`@KPJ(gI!l}P^GK1sUc*&$~3XA#A1;OX{IuiMBw6Df7H)kFd^Wo6l%67)W*d)sG zC2_q4TmTPbmJPlF+dFw?cs;tY4yWw)<%=(%!^`Pz_ssxh4rw^h`LK?qx<0{N4Z+0R z(!DoaI?u4Xv4{$2ZeD$2VPj#?Mz2fhC%_?1yN%m>trn*cxTCUAWPtY66xS>INF<_- z_yVb^3RPAbpiue#;CPCq`TW}FqK9l)?V;%+^R|0GJQBj*P!CFlrF%ubFyYAe;;CSn zz!op%6z3mxkR|;+!(esz&2XKA_@LKPW#1Z3*w|_yIgP3rNSQ-X_%Ql%WjD02&=_qS z?J`x&zT>)^dYuvDZWWY6=Uyo7ybeU5nim&8Kwj%`ftsz5gTm?+S4(ezjyFvPTr%Ez*J;F$Ys^6tc!e@Lq0VsR!7BxspratT{YD;_h}1 zQ{L}E=u(0|+A}^A`C$Bt{y^G~`zN$fn(|pEX@{NpdGCL}G5|GiRVKHCew6ROf4|?)a5X9e)IFtn7zy3_?LhfED*j2@d2)&2 z_WaZL?EmS`Oe&xNK>=Uw^zhff>t7y`;mdVNlirhoy8r3U@(Cc`j-K|I%KG`-|9(|3 z3(8>55|#X?J4f(=DpKYSeCvaskslytTsgLlCyyAdT$PEOU&;#u;sQPCFc71O{s&* zHF38ePGRhOC=hIZ+&fsVElELv}l-o+k&qyt&ybZ^(CmM znOtJ|phdc{w_fhrIdF-zq8$*Oi92bdmEk;X5$_pqNK1N_5G=FRw;xe{UUr>974Vvl zC9zMnX~rZE`XFv!P!tPp(r^>0iV0(p@S;K+4UF)}# z$*nIRnNBA4Y%1Qr&p9K|%{no{$TFBR3m7&^ppNHIl}v!X!%wpATGaLDbwZuVX`W?z zJ-Fwv#N8*zZc}9kQNAsl`$^+E`{;ANkY|ue>wM*u^;p=cp{Y&HJoZHu$&GK%&lY=a zs(Ng_neV|v3E}ZR`8pe+qT~66lWoTLdoolfs?b(l%M$O@F3=3pFD}whA9A~6X4qKu z(kcvN1R#1|Vl-l&GAKf&ERSX9^%}3*>xWVvrqSq)+HsmDZf9Da-!{qzkdkG+>%%*5 z8?$54d#_tTD!6;rD)OhbVHhG_7%elL?nJO+(d)#{3xL;e9^a)g4(DhJqm6`&90=r# z?zW!CRF!%aW+JWUdkdBm-P#}CbgMfNZ}OFG->Y?egb+hN;LYL?deuQ6G2I%gzqhwH z)pyj#e3^jl$yegC9{j?+va%8jda4T-M>MZ)xitw4ndmuk?tK@73tRv9#G5>*lzxHT zN#}MsmEg7l6qt3Q1cqfFsKijoDLBt&L=ov2Pd-mqFwp}yK=&;qS)|XP8BX9YBeuCf zD}JA_g?C%o+!Q?8|5+6d*)tHP&;=(tK+gEe_tJ%9abTBk4eUyJzjy zbmN*}wqb5k$+}(b-pMrQpqYh?;n(=UZRg3}Me;&p@9kH4`yR1|8WOYN>z_f++6K$3N{BZUt(^VrXA=PG&S4ZQ2eFA!|B^c#~I-!<>aJ&<=}xf9_N0h zN~iVW(dtW{p;k(R^eReydi=OnbIf%p;b~>@a=j5+7u5Jo&0R3O@tpWWArKXOSalF^ zdZsB-V)<8_cyaS5@GwC@lj(+?xW|Zj^s_8o^6f>d-5u>HF^3kb*!8UdKQjXL`rA>9 zZRm!ymM2fu8;&nX7lQCy0Xba+}?hd zR(F3Cl|f%hmg;a#0nt5I<1q6o(PGg8gPQB9P0vwFy`=@w<>tLUWs6&B*oXRX0sG$G zINbNh2B+EYQkHJ{08ont!c5I@-ZzH()BH`|g~XmjQ|F>+MY@!#T#iQF(xig(`vfe$ zaO+RBMp0vFnZM^X#ki;R7?L-wvPpX%llT>6Sl z8(qUo!Mx^YVv16k+a@~bVjpMlXD!v;-EIX;332P*?~bewvtD|?GYLEY*w%=uePd3$J?jIm}Eduw?CDJIH zvNpo>JqeYYZ`R1C5)mj{1{R8k!&9jx;!WXj7l~#zGjIZn2kw13vx!(81JKtJPXJaO zAPHxBs$FL{Chn<_85ZVY4T(0b?btPbfzHir^-w~dD*cY7nppKeN^-b zCav|JiZT^+nqJT)Z4MT7&z!)=@R`1uciH>Mjj*J`TrYz(SwI&smcni@8xo1f1717b z-?9ZV%oik9)$1r^$_uHKlxjW|g5gG5^Mjzv0JC_nHSSkZP} z&_R~&8Z@~q&TGPV$Iby@%?odluUzkvxwe4$#HrB*I`R(HTcrXe7Gpn$Sl{EGaMR7` z=gWRfq4N^V0}kQlI{@ppyYI5FCD>5P<7#1V5r-hp!jC1h4&Tz$-<@v1WQMj-jlBL` zbfBi@{F8D>pROLt3Rgvm|8={XoOrD67SbY+^e!6!;)f3F$fc60mCILwpgqtD`JNZU zp)nT1QmQu1m0-%YsW%B!pGqf=nKws_X*CChU9}>ZI5JzvHE8|vr5{WDQE%A|b3)h+ zQ8zy%EwdW-6E*tk*Qsjx?(05V4?B5uuBR<+g;zPZ=j>r}0a}D2U&rHJnIa#uYK(Xc zEEyr$IzrH*1_NFRAirSRRKxm7CHEsVU$~VQiEb$$_c0GlNLUW$a-)F22WuC3d7QS~ ztp~DopJ6LGG5znPi9%!4Z!AT!4HU$4n&H|SRd7@2<+>P$Nx# zuAMC%K2Tu6h{(~*zczeKgJm&nwrcMB=IWG^z|EXC(5uhJSa{jK=SwPTf!!!C&AWI{ zA0eZ!4!Ex%LtWhB+YfAya9CztT{3cW&2(+>Qt@=ACi6Qz70>lqOYcU-@R6hUv5n&r z>5v;$j~b8#&gGEZHljDKhB*(j&GvVU?KXfILqr^ez62jiE&ga6G@E0}R7;N1%s?Bc zMThz2MyH;|0wV74ib)y51hSrsftS4D@L2>JH-S4A*F_#(q4?XiuebfdJ!VAfWI5c+ znDD?N>q^~KNpbvvMzpsyZ%HuK?d_Dz5m=L}3Bw2hHJ@V!=vRu#;*tV(J00R>qMnnJ zI?LhbetOhVFn?$;HI|imq+&#_oKp-kP>iQ|PTRn%mbAyS7ut=wks`h~W8EW}2wk7Y z&9I1hO5p5XdGv;0vnL=H7up{S=KtVPO-nU=kIoeRka_lC2*G0mR-;$-bqX%y2-l3= z;YW<8MmV)?9~N5cKd@1103515E-hc1r*(gmsA~TtQGM3}Br3r*Z!&fr8JmG~v|g^w z@3hU=hv_G4NGwL`)8ZBrt*<3;*a}!`JwHnkY0LbXa;VgO&U-{%Q$M$M)kbZiZiI+} zcEsvPWel;IR|4Nw7Kib{;12qbJylC|!Bb%jDfS7&9LYIwqw?C>iHumhe#SsbTV3Iw> z^m&*UvtOyvSO^zYY_|C9rsUMAMI9K`o|Ez)`oWOroAb6@!%I44N?L%LG4g5tSCFcO zXr%zo>B5ghT`@2{W-Yuv*Eb%E>v{)%wcBUxJl;b9yE~Oz} z;!)#Po@PnG+T76i3HIY@D!4F#CtH}RGL*jlqHh^cS^$b-v@&(2_S z4o*M9evoM_e!sA;yjhy$VOf(-ojqdcTKF0#5Mk1kOI7TUGNSAFi zQn~oD+z7Vpx77%hHK^3I@_g&l zzhZeNF|bdUaCwoy-$f+3W;uAGUKR4M^u#Vb$IqnQ-QJZ={_eZFlDR-H1zj3r)l?DG z21X+0P~dc8-=~epdNAeHB(I;#6ZT-P(;I)OQh?N^)hVF)%O!iTbk59^M;N~V*8);tCPB2CssX058nEGS;@2hD&*=p`FK>sKFtHlv0D#&r=+e!Fs>r&h` zxM`KOgveM!9v?m5&0G@e{=$Lg+EUl)tRm@UCbX-imVKA)*>S<3(Aa@f&o^9B$+T$& z`EFWr!ahE-EbGe)OCBzF^CKN(XBa0eD4M2SPg2~^_&RW09u$QXE`v3;bmC=@{W$4xPIx7 z=o-Ca!DJj!VK{Y2?yAsp8l*KNX=R=1{AAzWb(k4_BYpSaWoo70>X2LpTTaXL+@|mJ z+zeGusfQRE^Ymo!a8c~e`PnI&Kt4O`t(cgTPn}M-L^jAptzL7oyXTKj^yWO?>AHGn zAaQ2WF=_p#(14>?L0NhxYH!V=ha3jfTUw+rPTj7rRp|;wJVcf`{T4s0gQ}QI#h>He z1;CF@t-&=O_3tOw>>kOR+nG2*h?TiW{0jMT%nQ|9py`!)`qfa*19Wdj_CvBa0%bN2 zS@Y*5U2h8aEg3JWJSJ6Wu0PmQt)8iwteLrSIcjU{K*D1a>$OxlbVQe!b%7_{txQjIx6b*`}m+TG@0GbPggm3WzVm(7|gjmR4={m3LjOWiA>dn7C29fTfXTq;hHRLsSgo* zSX*#M6(nXKnK(>4Vmlj926FT* zIQETp!C6}HNm!V#gy+201_OE}WP!dXU099HY;dTQ%!VuwGstkaW16_4QR?F{4hILC z>`Mfk_$$Z{1+SugQM~Hn^?B!%S=eGcK9yMJ8Hqf9LR8Xm7V?ApYAoim3LD|U++Qk- zjw$YaCD)QEc&q#5)n{wQmUPJ_`|V!u*P1$X7A@VDJ1YaUJ`#gX=faFq5iw>|J)Wl0X4L?m$O~fg=_LPsfl&vkxl?d2;U(D-58#vap+vH#0G0pzsNjXMgquuDowIx~a=QCL<@@zJi8&)(j3NSUko+92chv?0HTdv$fj&lQfn71`;ExFZ%oUJ8`0u$g z1^wU5m1_!0VxFuF+Fg!)gq~hS*ezD$@?b#kPCrKX`rLB0lbibYr>-PNAhxI`C+TSV z^4H|}=NE0^Aat=lI>uR_OS&|G|MSb+^tH7wJ}*1o{jUI z|H?pmFu>Vx3I8Lt27@MIIK9>zk_z|7q2h|$nCtaBxc*l2;+B;=PF1@J z1r`np50z0tbYI;lEF@Rw+>d{ez@N@{q3!;N|gauAj!GR5Ck4Nx@ENJ4^Q4-+55{H?5l2p<>xf^v+eEN{)>SYm?ns2we=8zVwj06ZcL$ZT zafxGkXQ%7jJ~7kRW(Fn)sKx6S6n{!Jr8|nm4cs+2EycQtXOH6LQ|*1wDD&9yDL#MX zzI(L%evO^AwTPtT#}VYJDg2klj5V+JmK9IV`CdY8k|}6bVxa`p36~o23bLj;m}6kf zFQ&xJO1XSoqpg1@cM`4Q_p2+Wk#Rq<>E7ap=H3PpchR0F!U1T$3c0&EHRq4qkuz%b zJBiB_EH{|PCkmc_KWu6KzLMN|9tbpC&-W^@-wj1>NP-|SP(!|M(_`%ke`p=2`5vX) z=*S2ZdG$g4KnJeN!MezEyUnx^!w5~|@e(D`K{{xjJ?sHewEzYX^7*Eugl|W_F0^W$ zi{O4(uG`f0P`(tuF$lFE_6(jl6_fhPU3!JfLkQ&GeYCXh@b2Kz#CtjIIU8+qtGk_x4tKzO9Xvn4e6p z!7GEpr+4PHiMi}m^qnvp+(I=*Eo2#WBSbKw2oErLLTRhfm2v5anXJh%$&T*SY@DK! zUF+-~N!HYyw6wMTn%W~|wJ=V-x6XGRs0~2@QB23fEzbnB;q=+1ZHhxteJl6@FgR8F+~HnOT)0`Z#u1_t8kEP6$VDSSJa5CTX-1=DWGo$EYUN z*|;GhEPPLOz-V}7(EMexlxq5k=(6clNJ?OW-dko+BixG_PJ}_k%4E>Bz$kl@uc9i6 z(QUQks!EwLHDINgqpFBeC9eKuH(u?P9p_a0BzTE_ehzhUpSji!ksPWA8WDz(DPS*bt{eS2!q_3LD^*F@s7I5tIot$M-e2m)+3 zgqBTFqhj~A0^YsYs#?WXV=?J0qzZdqBt&V#^bGl`eu&_wyZ8?_s|IY#5@9ZSE6`u%kq9&;I zr8r#19v%?ny`t%{Qp}nvVdPJ^coiQKACP)_oV=Zq|oQB3ilSwdx-pJMs(p<*~DB9U-zNm6W z5C!rPp*Htd<-^YwrQ!EI71g3LOz_%citv{Q6e>ZYlfk+sG@5vmPq#-b*E8&hmW=F% zJc>k+1$Yu%hB#n*4I3MzpU!%S3%8vx8Wh=6$7`&Uj7u%I=~PoqyT5w->&&ktalfIr z6MMuHBgEaGN;=kQQFGF3dBb#S8`}rnS=sRV=5Ue5z%H}<$XO!hy9IDYS<@9uf%5g< zMk+hwW083~uDQ?o+lp6hmD|-gv2)Vn^N{))oTjQh6nu=u*l?}xli-I=+w?u4-#Qvq zY;>{q{j~yujnk1EOlA5YD6l38qpT{AY(2_jy9UtT@sl}BhW@&e92NOCAx@}TljiPR zZ7N^K+L+;^ul4l`c+*(hejKSoDmlX34rlKi7bUTtDJ9luVa3c)BTC-6bEimi-c9#? zu;^+?d8q+Asec7_Nnq(>R#iz#falddxdhky8Vytu3laKR*_zv-o!3LJEN(fVEHS(6 z!vpq7qVO%KBjRqeR=UH}(Fg}04}@pbr$B6;rrPm&3>OmwXqgzavy*T=VCPXz#ttkc zGS8e{$WpVEeJrXLmDvRm&+?{;(lZmkX7V8P{mRgUXpY;lNSAvmWL42^a zg|;$gzyzmi=fqg1r|PLD*Cm6**w?;Ug~*lbLB|6s{f7wI8bFN5c3+oTcZMigkWieivJ$O4nqhfch zY&bsJ$a`?*5xuU_R4jXIf0fyoZTE!d52l=#wE6&VNFp(Ual%iKn z%3DabTCHNn>j2eXQ*uiy--d_SpPijJ6q=7n>j$#4WFJ5!r5b2GCxHSeqm68dCOW3C zDMNfvCX`(g9p4)TOQ)@Yc5g{gQW8fxwMY%5p75qKrqO_6g%kZ|zz5p`|3H zY{%gfVv9x&CsNUN0q0X*)moR!+srgIvR;;Kmm=5gJ^ilZ&oahgk}w)bbgOeXb1jqU z?d#E8W~Anm<(_i7zRSxh`rJdd%f$Vx=||lnQlZAf1}OCMSLq$z;j`U*PtzyU%EiT{ zyO^zI`yS{%SL}rK(?={rkMMEru&lMF0w?-SOf830Nqk&XoQnoT`i^tb+4zb2E#BQl z6it^DSGHZIs~ht@To2N4RvnWjfp_P+W9!*;We*KhI=0%B@+nJ=VT$TL6LClcCsBw zAV=01Zw5;{zME2SCDyO56lC7VS#TRIUh0 zh)jmyVyDK~g&$?B)J0i0r5H!w*Kzi}jV(CM>i(EQac(ZpCoY%tNe)++)a0?yjxA$O zbH~v5wHzMzCtyF_Q`XbN;g2NM=FDPntuz@9N1wu)j0;4a)j>>UwhXls;@fqMz+s=l zC?n1sk_j~(0Fb#xA90Mv*fCZG4FwM{Ir=uXEw7!v@hw4M6B9KoiPHsc<~%%FDovY-Z=4s z;&{`JAs5D4+(=xvE$N7LR8actI5!{)Y{G-HY^L9%7u3)wBVMM2Q@vtDrh>XN;@549vj?yTquq*U^YW#i@LgeWO7^WER=i>*4uGwyIC43EX2UoAy)}rrZWc1EoKydDJmX5(9 zz=1T}t)>C81qgVPjxO^JNz)Pro~kVfow0l<$CX?+acPz6k}Yb!er_m$4SXPp9=RXV z$q=9xQ@YTS6%qrdVzM2%{ymVP3s1bYHD zC?l-{8ybz@J=P1*Q46`@x;vpPqv30WoNur^>2|KYyn~$VTPE|%6#t8PFn$8&#J(LT zV&Xnm9?q|6d*LQ~;!tgksP+|!LezJM$$y*t;3m(K5RB|9mxIMmygHJ9^QtcyB9~uk zms~+iw1?!8|9+Q0pXkRh4OT@4;3kW&$sY1wfE#;&SByDuZ(QCb{?o_O6kmGw^pDz@ za{L#M@&C(5ei<@xANm`^VoZH$NNNGlsOP7XgIlaN`AUv|!=l*c{P`Psxy(M&IVHGCblVN$|A#0&^@D~ zh!~Kyo(fK@fJEdJkL{k#fGg`uAj&`|MlxkK`J!@ik{z#I3;)dhA{PNcG^Z7M#t>ao z-t76Zb9#+D5&ya?NWo&hHnMvsEJximFW^kWIl;Y3q|v3VGj|4VHM?N7p$|K?+sRybcV6ICM^2MqF0F!XON!|3zCfdX`rXj>PDgIARcyT!2Ky5VF~T; zr7ez&6IZWemc6j2pZWvNIZgtb1JLnGk84_mCoGSSKf^fV!Y`S4j}btjo+m)g2V-08 zjAhq!d4|kW|Mly;&E3VepdFf0r}(h9@+Z`&bOn#kPe$0rW`E=9L0d8?QlRmgOm z>N^XmZMJ0L7DL%2sb}K!?}+$N*Lm2z3kp~!A$Cv418PA5Yq>;c63H0e8y5R87IkCS zqQ4l>i!*V(&b9QmSWc&QN2w>HoccSGBsi0KvxeY225Ytn>1&P)&kqLlrq571eeo0n z99fABW^OoI^uCyNL+1g|$ula0=*;0Tc;@Ek&xx?J!#vM7Z{9po1wp?zC`2&6Nhqc# zWS}+soxq|vWI>9t@u~w%Ad&D#G>H zC_An^k_5j7V-7Rw&rP$M%7v7DZ?jaa`+AE41ozmH{YWE_@ z(yv&jV*VQx;ym1RCt1o=OZhh@gdryN5G>`y7CS*-3+^}qk=ELXwN6i^ z%uMuM#LhSY-|1yZ?yhfSEXCws@s91K}0K3n59RvSX6HwThBQfxU&;{ehBcoAMM$=UgIt`L^UNH$DfdIY_fV#b6$qM^Mlgptte&hG3Tc)zX4shG|nY4{ehZ=L6g(> zK|oZZWEyj0ZAO-}Bf~tJWo3l*iO(ejZ}b?9_%>ao3$wuIyCVw>PVE8O9E9_dS4E<= z57lIy6_$FYS=Hl#whqMSa(;8x6q7sYH6XBG_{z(RLZHY@@H_3VCa=Z*!A{VyZ32pX z_SRYOXJ(!QUF-tzbn%l8--K5~LEu#npNq@X|2P*=aX`>;)})tgr}uD^k; zU|HYb#A@{skI`%ZpBXy(C$MJIF01lGnY-vr>|lO5wj!xWoW2=Fy!vAWn-*;MEwQT@ zrzm$J%-T6xHR&nH*~iv)hqlJqDW&k|V+O45SrJTQwsW6`*VK>5^}kJ6>XL9+Ovvkc z+cqAA+3UH*${W)wi~!zfs`SM!V`#v(8rr?MPFiX?L#OA~>u!vpeCLj`bP=E`<&&WK z%8&iCl75;AX;S1&x=r8IbvwK-Y|t#Y>6YVnB;wl+(68Av8T?%C^WxVLrlS1P$T~%U zKGYUjb8?dA%5m-C+XFbtSx?jMB7JZ!Q7-GR2E2kC8}d?5PJDW|VpEreN-9bYP>&+a zh3ltPCs?Kcd@NHJxc!bkj(?yJ{xds1FB=gzUR9~PCPoDczDd?rK_g?0%a6&(x9U-y ztGYOx_Xp~p3TXi9QGe>qqG-uP96|-fymE5z_?DpeKe+(59xrB_ZEbw2b5_-i;4iUl z$$zMjdR^0`9|uS;8%6sjhdleXbssFJlciF7^`$tlk6-IF)@sl+GY1C?eG+!f)+8Em zQW<@^JE4kj90GzgaB}Ppmgm;Na(x$gt3?_FEU^6}7N%^!espaJGUp_4!(4Z|P$eiO zr?+bgDc8!8Lc=`Tn?uL2K}Ig&jd9IS`#r?XgpYQp50^gS>uTIp))DB58EH(J4)87$ zx@3NNiXt%6n?ET5-Lg#T=7QA^Y0FmCt@bh0ggvhjTk-Nt>EFw_>ZuE)fI%&$)Ha6n zK30TG|>8m~)JQBmpbHfOgfx)<6nj@TH^4(MkEIwH>!=cWWO+igx?rMQV_ zh2oK#W-=AWtC?jJi3wX76wM;c`Px-N^`w6^ojGY^A#3VuJw=oz%D|oy^Cz;yeV*Xy zEK@+{Q?zd2()jtnbcZx!4$mI04a08mYosNY+qIw@jhThsjbss()=;X(rIyEeFZcdp zj1Iczi5sSi7q7gLL+tPa>8^1;c+;!~wm#^Cw-&tionWGE;z$EKS4aPVmMQ(uR()oM z@Roq*!2+QK!-aKVw7mGLDOQ?%1Tit&IGtuh$)i}(joFoM(vj94-p!unW~n8Gv9PpN zib@dOx0)~aK1={PM`q_Yy7B#bvuZ*E*7~#VaK2TL7DKACUp~AY{6rABR;d` zm(JVky=4`e*n1@c_RB_QJa&7Kx{e3U3$Yqf@72`Y@OS($H(K`!<$Zs{7OfL^Y5bLv zCqn5)$vh@Ay+W!QO?aLu}v5nP2MAfPp zH^h1LS|<{OM3(K>4$UguaB~&>uz>)e!48$^>CeH}8x8iZu%^sn?#PSvzzpunyJ^!> z-S|R9CH2@h;Gv_F)2d;pNAp`b=w}da_D!FyD%%x}zv39@6c#mf*T28je&6*}rmoFu zhz{BxDL6Q!@PT<;K1Wa*#!^SWej_)o&z+1YHrUN9_}N&_+G62EO|pz(M$YDIZwflr z{zAvmab?fS(!Nsr-OVgS4}ZKaHkXB8lwy|JzD5}ux^rTY=p&W49R+)+REUIzL-!>o z<^oiZfIRU+QzaF;yI~eRU#x7}W2zMNlN!liEZ}G)+ooABeILUBS?GH`pQ+j5(<<^L zua^uoPAM9sCb*AG9m1S1s`Cg)%vHG@TF-KmtbIi9dSD#Wc@^%PX5{fVs*p}n`(L99 z`V1?w6vcQOa7ODmk;1~AVfbNslLnR`K2m*XjiF~e4iF*i3!A?(WX@{qkNHYK^{I7n3GJ^^7hh%szRyIZO?X?Fpm*TdClD zm8a8n>&&()d&wn4B++_4eamPF)naesl`W0hUI;R6p2aySohI|0tJo>zy2+AD!)2i) zMfuNEvA6bio^B;g7w)UL#&RO8Epn0@9nzn0V>o2DB<96QJ?cGVIN>(pQDNqrNA%h^ z{b02`zYKJvT~W9|c4Kg7^r%rD&XqrZam+kBKpv4xxl%$x;W4w9;z`sK;$?%r9idOd&pZZKO0N;Z=3aOHM+e4phuhWsDufTjLV_CIr9&o+w-bhk@SwfpkN0gNSLHLXzEu~ChooYEfE#T- zfOPsyzKwey70kU>6>h{O0TUe$cto$0e}24 z@d|RVX$_CKftaf8zWxG0gW!mLzwdp1&=?DNfz-kunmv99jz_p}uo`yqmO2ldV%LMi z=Gs&5f|=#zxBgcA$QD^WywX3Hhkr1_O>ho%WOGU2@z=|JG|pkQN7Sti7=DB+u3O|j zkL3`rXL&mo;PhaaBK_Pk{+4_F z@x2%C21f&#$b%Gy)&GW&$iD`Fg!(hl@by33tKZ1o-!>FE*99Sw))BmJVG8rY13}FxfAbR=|`9{N5l=5Jc4w*dIk3p3;%7 z&Qg&8lp}a;l%i2|lW$P|?BeOP3uB0;TCZ%F8fvWno1o~K5JId|<;+wPiq6IqP+heg zw@eanU(**4_dod97RLB3P@^VZJSqKR^#qxVf)IW8pC9=(lv>~C)I6%3Jn9}P;i37M z@!DpwQ+xe*VAw-Dh(S5;3GUt|3zE6Hc`1(TEnpj2tqukt4czsn{g{D1C9RQXGi2{t z;*_}U>@ueE$m99p3G8Xh2Yt)$-@&cw|Q3wISgtH*~6K(=75vSA$} zc*Wimpq74j5Ek>i{$#eXW0m{L(pcL6F$B>n_=-2lnmL68@)b3Cp+WOV|7k#}Y3EF%Pu;dQI9nlU)>}w(cIKQbu1YJF$EO-f)37?O@l@+7}AY-O! z8>*;$+IBm0J(ANz=EzYm%Vfc#3XFTVOVfQcG*y!ri)QWQ5(b2I+>JY<~_^m(^fyCn0FT?^5Zn{s5JzgP$Z#<3W zw~^;tw0I$#!<27e*95jFqD|f5G=)60arKF7nUWYWo6shV>28kVav%A=&)KvDA!_CPO1ik57=)Cyz5 z*{(kv9J(nFjntKRmk(*6lkn2q9mu2mV@0X>%epT-Y4S!6c?Omp%2#@9mu9dLMDP_z z;l=o*rdp_-)bjrVD+#S#`afhP3dyt2pA~18C~GQY%0l^Uu3y7gPaK7dC(7Lirvhr z$TAp#yNh(9AX)JMtnVf+E;tG{lQs9X1p=C0PK*39LuG=33E2xsvV8$bXoU9oCznRB zoD@YiCtPkFM^FxH)SQ_dk3YoSis4#K!Ms1+oK@^oN2_0)gLWK@30WTZTJxh7c$o>jYD z6A9@GW4pBb@KSiv@NtmaWZyTV1@b)6!rAMklr#kCbS^f@0inGM_} z2>OG(pwwxdq3l{2oxj69qsq5-(Kk45xp=QR(u6=4-RlV2kY<_~>-+Y+B$cr8KU2aiY! ztSeCg0ZZ2oq>OAV#Y!#Ko3{nto>K)1gF0sh;wN~2c@_ghwn{2OV=mvA!?g&njb9_R%MI3mXS=o( z6qH;Qq{?QTF}=m4TWeFlm!HL7BB{n(hKOgy-6J4Ac- zd@I|gxold!04kY}wTpLFhrVIIj~}1*q}V#x5Xf~}<=E~d!YnnguU`Y{=arU8kdOig zeDACE_w>}sA`$d|%*Yqcpw^=|3tP!mVAx=C8RO(zLj3p;N_bX1VfWH$`>Wv*P*p9R>Riw z4AxVQKUGatR_zk5zBeg8KMv{K0+rSyB!gb4mv%r3YTVE@2t4>^`$L`)Q;Z#}+b3&xhcl=P44mYHj16U&3G^U2E4R~QEz z5ym?0;?;gRNdEpY@{En#^4)-1%U;$Gq?AZ;ZEw@O{TsTRi32wm*WD;ld%k(oc~KLG zSiU<%(=1utswBz!bn9b%A`#`vgZ&@Vpy#rXp;m>wI69TL&UULA%X&g`xq{C#RSOhk z(xO-lNe>HGaV$s_2-+QBt8YPm{uLNY5HVbaBSkjh^On3xXe;|RT<&-zCc0O5Y5KkK zdKW@z7V}fUn=-rFO702!-D9TdJ)G?A9A#7j%dHv*Ze~{e;1jmY4jRa*_!blIT42JP zi5%r@b)Sr}StH9EsmmeyHy~Hn`KzWxKu{Kws(vIXzw31q`jzwG(kZ>4SZI{XA$gQiK>np=#!`J&;kN3PF^G)n%>BqKbWq!B8`rpG$`U$xu2PL z#o;xqZ~)sn!EgimI>qU_>(OT21SiCAt#vKtw-Z;ivd;+j)=DG>{S`TkHL_d2M2 zXtZpVNgA?jrXS)WL2kWt6SbEATi{gj>MZI}v2YPbxGJR3ZA|)MR{m>{xiEPM1Xl*# z7bH$(ek?RFn#gbFHn3}`1P^(v>;bC0^*biv(I`5K@mw6p1xbkHA{pgOm0sm2p)-aL z79W@H)jTKS)kdmR+)_bEeSfeKk;O=>4`yLvw$QkA`U#HmRfQ?(h1FofmCM8*n9jkL zf9WdC#H=gJCh&q9Y=8txPb|!<9bXDhNlE;dXBWJHr0_@zK4rSPFr|iahDRT3VqY18 zwj(AeQ$6U07sd4uxq7Nn%+QnFco>JM1{ws;KH~-JhBNC&SooO+tXNijdqj zoCOCds!|Z%u5x7d)XnwLbeBP-gid}aX%0S4Z>D$Y6j;<8S_fz%D&$Ln`2vk)F_alN z@v*r5`fnW##*G>;dbOIqKsJ+pITKG2-N4q)kJ?$L$q#%?zSlI`pN5fDD|?P9E%juT zFx*>rI_Pmu^$+h5tv$rb)80raC2>|m&P@ykW$-l2TXv9x*;J=IK%Rs}os`}x%k9e8 zC#7Zip^Q$9YM^M2PG&LcMGUgXCIP788zud0pQgEt=n0d%Zp z?DcA#XmqxM){m66?$Tma?<>uoYfr$dplLQ;o=iR{ zC}K2g4^e|x6Ik~sF;#SmDOn^-GHk~xGq6j#nWo)?`tO0rvTbQGw|udMv?QrfxAkMY z(D%kpZqTv^)Nx(%M7M3|Oq8(HQ8DwJsGFkLogA4_kq-}4ZtEmZrvuhiYV)DgXl<49 znpDC-UL52_6QxkW@Q|3lu@SMsIwp{)cG-X1HW&qi`lQUTF_n#gH06@-D9DUDOoS6U z)`nB%Bl)Cn0sOkY)wPKdg_@$;FM1}ErA~aB1E^_248=+wPoLt?ts1c#>B=JsCt=*K zEyVj)IH&!hBq=^MfnSI~BIj;+-04i+7gv4>vuUQwd%P^U-E|d0U#JOJQ&hL^kF%!N z-$-RlW~H*3j4NuK!Y*5Bmwcg=o(O>b zYK67S_vc-{XkP@Zcr0@AwLReE;(D&seJ`5RiTE*@%wS2jN;TT?0f`?SbleiiAb>qs zm!-xLr(fX6c=_puaej-+YCI!7>Fj2D2q{;uTn+nw87q3ILu7{crBzyIx?n7u_ zc`;gwtjccU|F*U9Lk{#g&lLL7IVKt&-8%-2u~8@CQ=QA`NWjSYi%ZF5Dzu7W)WKB8 zdatFzb5o{%t*kH4Zp=Z3;q=wVX~$jUT-5=7Nh;A~YKn`pg1I@|0Q?s3F zA}$?-okhkspN0LFiMV)pW~)%i(mj@}8U$Rq$)H2!^@WHU zWTKW+fL9w20|d3G<#GP}6ivyq4b(bKn;!p1$n(BoKq}^(WFTYU>yiT%J_DOe47)xi zjbi4jsqqi$AUCNy7XeMKsmTor67 z0A}l#Qa(0|W_C)(^xGKTE@=Hi=d-PgV89l+n)=V^5Lw9(8` zPK||nMpz9LJ9h84brEOQLpcV;kyS;}$x?!96p{$b(_fkn$MA64tssFS;QpRLD;|my zCbP$6d%kqL|3q&!!2#u-?YNWMchvlI0U!?rlEDi%wbkPd}~mGv}AUhFA`@@x!UBijHT@0tWVRd z8iu0SAC?F;YUe{69$4t9j_;fCko&7Cxz^}oyPw;n@YBu)X1iC}b~|oPPN?_nJW>IBWbd23vrMX^HWo{t~b+cu_# zEJ(KGvGm^S1nV>rF%hIod_>BExA0q+XI@8QReN(v>YWu`Lk3PBUzOmT3Luz&G&8Q9P@Gh z{rjafWaJ37GZ1en-xL zu~zzrcMkO}kt%x{IvGURbMoGMf~=N7{tHhTnC^pEIP%BBBb24oTDMh-idwj%A43+u zCr7^Gyhn$gC|d2C)`w_!)48$Gi5vT%JtAb~tolWFJd-nQBQqmJ(A?`dXJ#QFYH1)5 zi&*id9ULUTd0B*A?%)QIFu@n--~<(LI?C+?{VRg zVC%Y)eNH9`p)W8om}T0yY}ZVSA)sRTiMw486rSUKf_Gb6k4`J$zK+6#{U(`1_hNWF zL@I5PQ7G$sr*eEdw?1A?KFub^-gU$a{Sm@7e5k=?PLd288@&|2)dAwS2nwj?Ygcf! zmfy*uT4H)QyksaaoSHINH4tA&(W@1Y&+$Pc6p!!z5xce4N>*$pGCLGWP#`55__m`n zbon*KhR^Ep+*pTNiNZ8f%yv~02{bO{qv{RCVp?U!3^vW(WvKB&`;FdWFO)z*P=T{Q zGBp;^ulYFl&xo=5Z3?w6vnk{<<}z+(dU{_}bqlLvTMXv&VX0}!+W@3A4OOIOj+}3G zR#;GkwAyNvEU5LTZ>KYa^Se1$ZY$k+o!b6fU@CU0zei?Bf3t+c1vUgt^%2}lpVvy) zb|gA=ISj`1%ho6%%owB-2KITL>$0j#hScsn=cMGWMmx%{u=g&-zr|1McP{`H#6V>p z=*r`QUz5q~*S<)2($Jmz7{b@kPXePpvR&>M&}$AK-N*pDoI+eGyXl3>WFob>sc)UI zbNf>C9Z32hCz$`REj3NPMy2L~<}T&%1JBYLhjY&kwVJmMla{q?1w;m2DyjM6n@d>O ztyV2~cDAf=4X#86QnP$k?-^&{SokhGnVjqwTRC^5Je134J$0*CUDhdq7Yd`WB~5~Z zs08u(@PWcY3+dk)YSsPjjCps#(BPt*ON*FIT5fAWDE=ua-SWkU90Ze`>7F30nKefHidkEF2G`H zFP7-xk--)ppmcp`L@QxyeZ5JopR-k`ls0xR*n!=Uxw_P|BBOG;rYNIwmm|xc>*Tce zoxejOLe2ybTS)QO2iB$d-fSjl&bQ^79iF;mj%`I~3O%e4wO6lkA^`+BQzlNaPY61y zUZCe06(G~BR_@azA9Rr>10th4()$$1POzoIR~YoPQpy=56ud^d-s(t%-Wge0<#|4H z-Gvplyz9&vfya>$2eQq7&{-XGiO1CWTXI#z`|Wo+#vT4Y=vb?B)JXqkE61F}-*oKU zLEpqqrD{15d%sXFdBbch8F7I=`mI2Jf01El^jE4IN}n|+j!WHmaFB1{3_{;ACz*Ff zJ5BdxhaAJFTrUB#dvvFPsu>`Z>=g(l`zn#8NHg|VESa@<5C|uJGHS1;{|1j{&(UuV z;IVXS0}4AQlPMs$Lfg?F))OBGc{V*m(9)0Xw;+$(MJ#RFUdbra4GII(?PWa;1%w@% zveTSL;XKvLar7};ui>94GdgRi{w}-Oc&ll-`m9*dt}I*zA<9ZavaR_H5f{BCDb{Ia zPplF0oaU2>qk%<^E6``=)!wMC-(I+Jh8X;^sRVYQf{^~a2ZsUp)iWd?Uw42cFLw>l zvi;!EuXw25Mj2$}{59n|E|*eL;!G7~$U-5SeD@{q<(jU|(Kgi$E#griX1oKLe=*x97A(URrzcRMT!4+y7Gu5Yv$Y`fczsrni?9Za@Q1)io;+v+|k*znD@QVC#<$Q7g+LdZR4gtw`um}~?A|_y7>cW)=2;(qTIFwes48EDN zk(h;d-mY{@jdcn~qg3kdYw%I8eqBW~TOa}(E&UO*)~ZUIrB*!jxK*t+!F>H%bCUac z93d+z;uLB2YB1d>0y0{<^}0fLFIiOe37S+?;cH1Y`;&yMEuNUZ8)PAG^VVwV6E`{;AZube=1MwX`yovu zwz!a7cb<#fchTOf^TLM(u`!8|L}0GYc%c|qeZfcaRqu!Ru{mWSU&N^i7?H8=3h6?B z+gZsQuBU2ij~l61932eyWiY7n4rNBfCXi!)=6J<-%x6vul33}!p+LXd3X zp=@q2zko|4RipJKjhUS+UE(F_8q zCzlQ1Ryts5nX9KT6Ip2gB6xkPj^f3$mB@J}X&|~YGJCexlaJGNPAl^-g0}>FwwXpg zb&O$R`w05V9~9kgTIG-Ff%1hQc_;(!=tc`-OZ!Vdn9-=I43o4+^C@l;Rr4v#-WWm8 z@tO4*l2LYl<%dhNx~9QpPerGI;;8S@klwHaZ@fZxGG{}Qs&PKQ_(&RJ;+NDa{z!$r zr!p>w>2{*}t$Mrk z#T)TK0#88Mc-xzb2x+&;F+br%nVlYuhAvIQr0Akco*)NwMQNQO)XV}WbHkON`==xG zOaj5U@|NLsgbRhzoFj|VS#Szatnt|6m?xd7$2#@nKsmCar6TJwF{QWm_P#av6HikG zyjaO>=xYm!Ly(Bk2#;4Ph{OICntr1o2V!?HGU2#iFrkoEO%AkZ72P9Ti33pi?~`K5 zEwG)vuil(Bv{YXLOYKH?Cx3|1T!}eZ)1D^3fmt4OL(U?Jm7t;9aeb;J)-gpkf9cU0 z?W1%dU)?m-v^c0&(SrG#)O>UY;P|L0r(ykGC66zV1!A#Q9+r&Br)WjB}b2BZJ zIMY+QVs9c|W#|Z`Om~WXPX;R(KC8DpGBCDLH@Wdi)!+ z`uEFKaJ9AjX)t8|{ja~jYa|S$A%(pmu9*MDW7MVtFsNkY`0Bp^j?ZW>xMyhWoXzhy z{An^>`gt!AlrFfZcs=&N|Mj zkMeVpgHL^ssAUP_IIY+dY=@s*kK}?*EyN_~$|l>EIhKHmD0l7Gj*%5<6dSDyU7A(x3iVmWHp|P>XS4TJMr*f!3>0mEV%JDfYa-P%vR6s)prfgiF z62oXc-x|ON%5m+4`qLNp)JQP)gS3Z^79#}gl>+ADm4Q3@wCVeHLT9eXmebRkAPetJ zV8jmS^Bf)~2p1?>+KLBxp&UGKExb~|9m#I)8P+kX^Ep3n&P3^udg>zD$#%JSrq1g` z28zvid1iufeUr9!`vI8CA+_5J1t*LMEFy6+Y@5Gy*$;!!kT1itLRq<4Y-0Jk&YH8c z`a|4Igm7Rd$=8*&0p*?#2+E=G&xqna=Sc@q{ym%0k2FonSV)DgBgYAsYQ#a--MY-cMeoQpu1uy?*)LU_ID$@EuJm!6PGd%(DtI zX=!PiT2-xZqp?=e37*jO@x=f4i}*w%m%G*8YC8%KY(;7@q(VOKCu}R0u`5Ng*}PfXBBFV{isd$&qlKg zsj5ZgE7+n)lAk{FeZc@}a|J;}y&_MdtN|VAf9|*7h;9K=}7h!MRJPAG}jYA-tYoc(}FnK&N-T%YhTSrCRegC71frx^l zq9`pPEg;gVba#VNBQ4!w3(_$(C_SWfhk|rS3_T#t3>`zqJ)@7$BRrq)`mB4`y6d;r zz2ERZeexI|?K6}6RYwwN0xb6%5gK-Hf4jQ@Y;`p^xU22>v&E{Det@r>F&KrzV z>a#B>@wa6G3E&$kA;v@bqHc&B=`RtO2m=SlGL47a%Ru9nu)6k`JmrHKNz~sBUp16N);O zVlD1<_FcW41D{);fsq1H1*l^r+PP%?DJ)uKdu9C5bA_I{7Mj!MwZiALf7LO1(D+Xs zBU!W|U2nSHopAc9*U~NNy9`AtLM7#RYkGJ%T|KvF6$dNcnnpRK>>n>~lPvMO&=BC1 z$^?5EB@EmHHKZi!|2!q=bjd#nE4svEvd2DICw#epk6+h0?DbwfS-7gfm8rXI9a7H= zaTQ`?JUDQjyChHaq`b70oMvWK3rZ2#N9$nIZs8g==sCjECO714D1Yb7nbaha9jS7t zJhZKcDDf@5dhWIqZzTjwk(Bu3P{}r``W- z$$q=0uPxG&$5zI|F-L}$7;vgI8Ual8;YFQm6K|Dd0Y)v^KmS%3N6i#M5H`j2I)D_?cX zbbQztxmss`K%&2#$oA~0j(54*Uxt3(0TFGjnAwoI8|&^?5LlyUjjmc%R!CWEbuO^x ztinV2!PXs8bMweg@JFsQ1V!c<8sy}i|9KQ(g0(%trIUcR-r=vV_{Rw^geyU49MwI& z@@GUnb_%Ed;8${;Y~}jfGa%k}->pF;T-6mD5TVgeG5C`G{p*TEene=`!su#v zw+ReSdU}Es2z}2h0U@7|l=0(RbT@1-JYRSs_wo56UtlE9wTk0h|D9rF4^l-8*6TF)-SglO+M*QB#5EPtlJDOIq|poHsXM#PhU;1# zGjvX!%L2Kh&6qP*?Z}yG_w1P@pVY}We|(2dR4DbE>>RD_bae8-?j~t^8dSHa!ToY) zr3>zpZZ~BcDcPgjSGXEb*;KU;)nyT=k9|4B-)kyZudzSQM$pY$nt+VEGp6BW;G(CZ z0g(m2TB0+u^$Y*W>}!oJc}{Z(=uz?9b{GHRM@?=nDqLq&xCMYpxH4*{Eecy;k>{H> zw(9wH-2+!le*Fd9nxsB@K#)rLj|8cpVuTt|;+u;%`evuKgDFL?65TL$w>E_=-pnM| ze=FPfkc4-fgbVgD!x~%gMybVFz?8vcrSEU_;LiMipMh>Xj1`tio8k znmWx2s)vPPXlMx=>}!=BAe+B(%0>U5WBUcwwa<|3?^G30lA>(zc~E-m2OuDcU>U!a z#d^V>*4{I(6##PY7XmNI|Q2tKbIbL%jlfuW3^!;+@m=5xg6}gyT(qsBG+?yQ+TY_GRDlj&lGaJcT9|{$|Q1ZM| z97E!^A;wvcB!u2%GP>+?K%>Lm&|Ff^Rw0+J(;*-8aS=Ew*i$ z)I5tXO~W_7oP+P?<-d9NZr;>|npYjGvcIGE zzSxZ)VSgbap6fJm=Z4asDwEt|`k8g-ce%Vhe$^*$WCwTl`_|&pK;^VHNpB3V8!X7 zL$I%#f)$eA99F`{kz*9oLI{eKo@6u~*iX}@YkbP@7!5ac`YPh!t-hI*xHP_B#@>|O zM;%V*XdGJH0y^ z8@(zjbnfc6x4QZ_PBV~P4;~enJ9~kNYCJB=xskf8LdUjDXK%6bxm!g6d%+kj1zu;n z#hx2aOVuJTjLI^E(cQY5d#im*Kf{n@`ru7M)a*M{RBdi0pRgYBp)YstwG{1=1H-s; z=6FLpk7%KT|1^`p`~?az$~MIKab^qR#gF3n1kg>}tLU*67iv+?zfRi|KDZ!Z5vAcH z7{n_*^aWXgRX^xt7&^>nfUMr>@G zTSP>Jx?H1BbFF{V$Px9upF2apQ-CQsq-}Gk*s^o5 zI^IY7vk(P${BuU#ysgn4m14^=N|pfP=%v!?G$m?j*x+a6(UA&S|LS|*&tqBwsTwwt z<&j1pR~A42uILA`*75CEa@4o1d8WsWm@lrlG2UeLD5alRsQc19opW_w=RCR#^}ZnR ze8U+8Wnb9`Gi&0R^m7d{25P(pmB=~XdZ)k&05tTvnfUR9jC1|bPLGt}IR{-vOuzJo ziHt95auD_?qI1b4*Hhd_l3#m$+h+qu9H9qZXqp7>wP}$!$Kq6MX9mqU%(v**?^oGa zf0~q-Xw)#T%=j)lcQ>AmMiedSG4hZuBhXl;7-e|8t-Uwr*#-YPV)U!k)W|%VNm=nR ziWe@i_0OI(gjb0C+5L|etNA92uU@`$cUb?_%J@csNSsas-Jh-3Ff9UOFWw)uZ<$1g zQi)ZOV(}d7_UFR%*{ppzv1Z`7P@KKeBzw9crVJLf=kR96Grafx?>-5Cv=%xx(1mU2 zDIdi#U+gO3KcDdAvtVw6e2N6Zkiq zcKyv|%Fd&2w}PVX$r8c$e0V?Wiv^I<;NZ}*!N2_72Ilviuz|H`yZ6=Y&!-1dB97(4 zW@tY#K-8W71->@F%#aKyDs1ip&w3KxzrWyu{H9J#Pp<^p5-Mk?(H91*=fP%%p_VV0Pjjg4cHP8 z64E?MaMN*IM$q|qU`xmJEmA#rAfu$L9BUt=d}1!YFxNV>57e9~^*Z(%MqJ1Gc3&WWB40b{qc|)k9YN2u<+Oa)yzh`=wj&`}+J2sBsVO`k%0F<_uPngpla$zgd$xUp&j7})lPPwOK zp0cl+Bw++aV$;DSIhNa~nMGv1>$yKifptg|MC;RK8fqdPO!>{IsVA`I&AOx#AlMhZ zUOJ~)Jwc1?$_~6l`aor-lH-xq18XKo2aS|cw@XF9F4@SoIA_I!({}d{h1_dLa5*P5 zyt#ObaSGY%dA?Jc9eKVl@Q4uNlgc{7nlW+>>yV|^x#!3*Qr)R`Ij=Hg(~7r#u;S2V zkSDK0+H$!_G`0$o(9d*jzLGGF|LjY3u&`YWnQ9TfHyG|b<$qFSvA4M#ei*nJl3R12 zqcu9HW+D^AKu1T%X0w(FrK~)uHGU{~uckh_Eq6ld2d=I-8`uvAc%>{N+7}+gIX-L1 zl4LXAeBk5nA6{DOf!At-b!-X2F4P7~TYP)r#J)13z7nnaqLM2iI$B2Q0n10zGsk{+ zferjhnDDvPKRur?oBcWEk83`Pv%C|4|IIR|yX6w~qK*?;P7*;J@y=-u3CZ z;_nat?Du$L|NOl49>$^)L(n82>%xgS3Qi8a;&7TEshX5x_uYRQ=FiWpaD#3+!~|VD zIl{?NYx~bh`0cvBo+JBj?_J6yWYB9VLVt3Ee?6xDO>$P${MAIS6Kd#Rm;XDF|61T< zll#9Dc|u8?xcpy<{I`bqLxBG8Z0##cj+d>}zJJs+B>L|4{yH52wGH-WYg1Pp6~*}L zm?B&5y^p zgR`<1zO(E4OeZvL*?;${P_5HBkwil&UE98@ki3-Xspe}PzEj&eX?x|;fmR;LkmbFp z0UyKeB0KqYc(tLqq!dpAY_y3|l*)H7IZDTNvX%+HH!H@```h>wCSO^vDdxuHC04vOX_!uk7kCao__+aL;7@18*Ur3#Gtkv`^C0 zm2%qgO^`9LU-|fnZ8n>zJJH3Q+wNhPm6FPs_aQdtVOP82Jp?Z)g1ni>P^CXCq7|tH zZTLJQtXH&{?TGd`f8xDU&uzzrr~W7tTs*M&M)rP7=T)yCIFqMBPEEG!yli`8cFX#F z<5BLM^$oY9v7noQs8ux}A`t6h62DO}f$o8(FMe$A4fj{q5Lu^sEU(gHU2v8o&wBEu z>1sy?3-JxQo6JV>Rmg?g>Yday@len|qYT-g4LSu@Pu{rFX9V~E)$W&EejyS7k zt<9WbkRdgWB}!fXB*gu*yDc+3)`8>al9>fRvEaS6JQ$OQC${{g5>nl`9hXvr%~$2J z4r%S#5<9p+{zLNxARVm&@;OeMNbFd{zO-AQYR z>wbEIc$>9&gd(q4;_N1a4c@}`&?re>G5iV8U;C24A{7*Ewey(Xik!9ZfSF7zNO!GR z=)W(5czC2ZPAo74ueb}oFPil?J$?EtGw$|dgAH4!0bG;Q@nm<1yZ)yq0-IUk+_o1~ z$lqdjkFo>FkF`URhc`8+SIs%$K1lg}>%GHA@yNYLwIyeP38uz3clo%I{nMQLWji?w z0fQ1-0Y-34>HeYef2&t_Y4AnsZ?8`JmM0ES;J#s>{mQREt9rgv$O4()U8QX;DJd81 zqudZ#QI+Z7R<<=75Vf?wHRcuCKNcH;bUtvsoFbjJXFSc)I`tt}fPFUMHk*uDe@+Z+ zVZ^1tc&8*B5n++8*Vc3#Khka_G8e>>nCw|J10OYGR*Fs@5=SOcWumM@R6Wd-BD zb#98qhEFbAY13_O@)MaO*xhRC*f%3JAlJwFF|+QSiM4b{+Os&Yr~A#h2Yz8nmRtUt z=2rZw$!Ps|xEddkc|%N&*q8O`5r6hc+0Mp$C0Z*+Sq@~22Y8(WP7N_cqRv2M2XBzG zM(gI0vj{qLTYz@=Qc0hkmaQ@ya`<^gX@!}zovZJgxF^~e17S7&o= zhqK#L>Tgg_BO%zS^VeQmJC)jWFu!F7{?km;`8t-E`hi+P75P#-4RR|(Do$g$==ymXQYi_Gyd5BY|n!R6<} zyp(;vJu@TmTEv)Pho?I0Xqy}rpLSoUO&>p`5;?eSoaQLs5WQ;RNI5>3ztj-5nhKc= zjVW?t>+A4*H#~Y~Qdax&=2gs*NF`=Re>1s%aUf52rrk@qy^lCn?niD-{|A8A;iHK@ zn!HJA8D6fYNRbYQdU{~DY)j@!NlE!;X|BIxbUz}A;;@WQDRntG+{>MEi@D35)}dMu zJTX0%9n%B18E$uy0EKZbc`VrsRyn0nZIUorYL$e3|9-zLO&7b&c{P!b?Qr(A5>ASTt=7=-oul;_oW=FoWpz_|Dg9XlhA}-hgNB6$tN()mjSq zhTV=CORC4{opJG{h^Z+fB6hQo2UhkRd7DDS1sUB4E~okK*8Ky^oeI?wr$&-&nF6Mz zXY(t|Qz6t+8b^n-5T49?(;Y6q=JoKfBW_y}7f9@t1&qMjZC3cMrY^fXxv+KkaeB~p zl+(uhaZJjN`4m;ym{cNp^V%}NCq)HgQi@UEiF|o;={m5%ASLBUU{z_GeEcY-^OBj=R zNDdEOMhW@|o%T{|l$EEvl(OLh^WwtqBmzV#*&>^31%K$s^=-8oYUf8YH$G%fUVOz z-faJHfaDR#ryY#&?D0l%X*aJ~G=}CmZt})z%xb6_ayw17IkQ_&PRp79o)m=7?S4vcZ+_rcZ>n(3K!x@y zcqJbnUO-!PnWn~b%pZoi@fL?!(o@W!`fA3=i+arx_czv^qEMjDXpqi6OIM;i>Ap1# z0iZ1E8Z&j4F0(-b0s<9+x7Kza#I34l9&`W>c=&Slkrs$HGb++Te?#PrS<4U5UQWFVSce49M)9qXxDYchy zryg{Gg3V^j7Nb&qIoieqiH!jy!8hNgP>Wf*Nyb`oR8}PPXv|Ny6z^APmfNE!mfQ3x zsinf$TJus(oHy_1ZCbfAWmgc)@)X@^`uIXVbFikSwSUahirIlsP#Uod*W^B2Dh^-U z_mneSH0yj{l&KW<)3Ips@n)!LjD@#4_e2FO$F`cr0x(VPlJ26UoMF3jkLuI0d$UrQ z&&0Au7DX{XuJGg34NoM-y1;b}U=}j(>;?%-cT$G|Fr2tHGom?I-DfL&RQ*b3&FD+w z4)hUH51&Nf-uU>qk`7Z$W(fuy%yrvb3=XB0muO~?ir(4{%uvkBryMkQUvj2q9eSN!MWZF0AtA@Eo}1)d|7htoa|Dt$!9}Hft6bf66A3z( z$A!IqJsGtG?0CxN^)os3nn!*CuBKTFqeYLz0z5m0#38E%gi5>}Evrr?* zi16e{DgF|H3ap%c($)+)42tSuYC zlt{Z$_LT?j4ifCnDjqS)#jt+a9eH99H++#cHayn*lwbTCwKhRz*93QJaQyX6Etl zi>zW+Q!}qN#n^HKQ>eYtN8YM<0;Vy%35@jY^LrqyF^omg#hMZ;;GJh=VOqdvYQ-tz z@rC-zRYjo#m8u2uq?!+n&Mq*)lWBbp2g?PKc` z&az8v;<#`YD%6$OWy^K=u_p$^3B%rnGG8>>Cz+ZsM*13Oy&5w_I=fHEHrz6X(_!|= z&eG08!_a7)QP!&jQ#YwmekiM0hwJ1$PonwfDsufmbwquJbQ=39%W1W00CUravY-5l z23C$ACkcYpM^T)O7&tt?yv$#4Yg3*#!x>eAB<`7Mkv~5dB7b&Hhsc`e>o`%;zhfr< z4yv8LD`fECiAb~mzwGuCz5$5B5d42=#2{mXjqVf`oXeMSBk#*vbsW>WywkxJYWK5$}-k$>N$Rl zT8}{;1abwjJsLIM(HR7lk|sG)06G>g6f zx7T4Tz#291)_qS*xDd??;2_WeK9EkOGi&SAy~%bNcj6PC_;d20xYg8G^i-{MA6--L zNh*rFpVSneZ+s_9hZ{=pL)**@E2F}%01N%t6f68>p}(w90h(S)7!+ z=Tu-r$6vvkEVTRj2I*r_VKNC7E7C)ib};yN0k8qI*AJSwV5RORnf(j-dBJ>zbdIy> z;8(m59HgR&zEVB}DO7xX2{0{u+s0}((VWWsavtbrCK@zlWIKxEoKBJ-jli%4;`vF# zdVJE7rMD{VDtLQOLVWmhxLskvbAgFO*%=MP8Uf$Vnm1m1|mN(LuH9FZUGua_XTmg7BAmIurDF1-8?PIAJsvCd6}2Vav>PVhL)vl#d+M(9;~za%DA;Fq|$Yk3nT36PNT{G5}Z zYp1J>EXT0VH@)?rPu^AKfTx*Aw~SU*7?0GePb~~SR!)hZ9?WcfWGg#qPEExq)0eA% zw_sML;9&&Z#ay_k2Y>)=!_;IoEFTDKEH@kdqi8sKX|0z_tKJavyJ{pR87AAN(D&Kx zPJIWCB#6flv^D$CyN2M2~NTrgJZF3?Uh; z{leTy4L+cSH?#mkFdcW!TJma9+XCWfp8V;jr)Cdl8`l=M^fk5T_kI|5f7E*m@FbgBMioNEl17o;`h|7~SrA@mk z{fUOH5Y?hI@*eFYC6+D#+rhl~3O45axI(@AGoBN~1eQku$YK=Bfe+|&q5um?F-Fx7 zu05%-O3m?^G-(Nk=j8WkSL>YPxMFoL@Y2F$x6Aw+Gg@Y=8o+&*UQ#?WdsiGUN`6uR zy$=HZU|%S)XwS3#T;8Z>$Azgf>~Q@vSBkG;u>6A9_!AV*X=V1 z6kx0OQw}8IrN;r@C!Ahgt5dT9_U%I&zgMB*yCqw%t-`!~$1cf!P#sw!7n?^j_bK+T zxAM9wv8S%-g=G3m!IR&?0(}yK19XR4KCoO?Z5^SJrbAnV10qvd3UZN#E+b)fZ9DW} z8aaC&_F_pCrtm3*I|9Drr|XO^ikR}-HOeF1zysAD8C{N~yI!@uG`nJEy*F0XKAq!c z#p%)WO0~*k2M(D*!)V_|CC_n=c~C44aYlHRSSc>o!PlhDw7DD}Kl83M@t=;HyYnD9 zfhYmVnkaV>)tEHLIvm(q&3PFySXs3D+&uC+otjJl2`}ZAqZu{At(X(1yTLz4vHx{T zgg#5S$KF=-wAonv-U}i!QpHScro-niiVDWZKXvEGx}iS}1+1;T&ok4}Ld3=#b;PjE zbwd$7-%@sMCz+y~QJl zU~rlHwET=GnqQFJ;xor8uknzEvUZ?VRr@^d-S}W;#IhQ@MQ76udQ;Z}2U+%qt$$I- z-1Pu5-}&x6-p@00O6A8~aGk9jHEDm=p7KQRy2{3>lAXou=^^tRc2%n#rvi)N!(5Be zO37lYgGOTW?y0woosfkhzHVKBOGFCi9C8bf@o8#lMPSJFDtZueEz&RPmn3nwakR#u4>@I+lm z&@H&Be$)8tpZedChGV;Kg}%L<^UzQwF06M^eUC+Q_uI{cre^I8TFXHdZ{$>t+;&6#^$39~GD3blhe9O&R)G7sYR}9~R9!`8|Nhf>uZ+Eq z*P8gt(X2R#pd!ubQi3|*0?p($k3)BH=?S@ zg0{C?*tL}`SpOmnk~7!dk_JbLrBy52EEFI+}*$rH$$fuyx66)u?J7k z&{oogGPFYa9Njz`hLvR6Z^IeRlW<*03-LE4vnUGl5xm7BY33JEBXqh8r{81GNn0*T zy(**&TbeNHAW0m*c?g{eFJ1raVomgJX|$lnwSC79lJZ$AN`jWLfXBK@VzK@@12a6c zKddfe$tScil7(NnC|9bb52hGA1~p~ZitJ=8dBW2wXZ$jv+nj_kz0k!(z6ub za5ieOdAKS0)@Tp1M*a$ZL_K?&Z1^tseOJjb_no!oQk{kDdAD%f;>a zAh7bMuM+eQi!5HCf!#4&YRUSMC$e~dD#>K2=Gx}+~^Fx*vVYoX`vRElD! zW+yFeT1NQW8O^cbCD4sX=i<$Puo!DAnmb2Y)^eHzxiv>SnIrw5GJDo{HPfK7w4(3$ zdAt2shMg=@8scI38?AxUhotzCsvl7O)<$$@9*cR}IFH6(Z*>U|Zdi;HK`POU6rcjcz8B3@Qi z?+ap0qHW$J(FUR4eWfxbRpl9{BKP3MRO}!Yfg@kZWEA8h>MZQAdw4_hnlT(T_A{iz zE`K>fHs)da3#Y)G(!`qr5iF1o|mPVvMbLF&c17kRJ;Z!DD9VIDjHc>H56x2pL()fN4@MLmsR|1 zRLq|?Q*y_4y1i7fQ-t5`p<+IdwRX0i*w#xexJ0Lphu4?1d|iJ{>2P|(kVIFSd3-@n zhALM9qpIWv)uQAB1WEfIK!;ksr*%V5^-1pp5b=rxkO`VVCv@*0JsM0V&d6L7{XiHF zQNeNmMA7XmpDMd3XF5?DKptnDzxjq6FOTvA6W7Ti^XYMxj1al{ zS!NdOOo%9oiJHR_2hkf=sP4fC0xm>jDapy{v#%aL^!bW=Gh~C#G?_GwZ6PX{Cb(=Z zBB9+^aLj!#g`0DuxsWQ!WpXNzEW!iQ5L7OK2DXhb`u)wg>m(1Rp4zFZ%`&NKFO7fl zJRB6z`YE z^1}`*TWf^+%&s7_+txye_<8;(@eJjHRtX4C=sm@5zB4mZuuyKEgjLD|$?^L7o|$V^ zM4iU7Rq>h`31j(?POGj0rN7c0RybcIiY84z-xp3p0Qed}ct3T30KGzd}K&bWtmy|E+4xjDR=bpVblzYPme6oN}=X>JIXm3@Pz1CmH%HIm;hc;Hc+$9rL zFSS{AGvgBXhIyuC^tz~BCbM+hUrg7aek{h6nEr{vpU%b)n->L;&gj`U@e*81tHsZb zTl#;PJ+*>oBUw>oHi;2G-*fQFw#zsC|G@MA=O>>p+&Ao{{K9ZdIsOMv`D_+&sl6{V zah`aWe}9z_*x`gYdM6Nw6Fx4`VNU>M^t{EhGS>g*&3^`Nf04w>N$}+F>HPf&rx`Ff z=c88t|4Nh<1=f@3OZESD;|m(#RZid0vib@3|BsUn{=o>eGZVqTSjxYj?DxSVRC_V# z*5!fkwuZky`PWPQ^^Ye2Uo#8-al(W6_ak7bdDlOyeG9#F!Vddqfq(ynfnz?L&HZmT zzVaEo%9)3SBe(vWL}&oMi58Xf3Agk2?f#zI7v6x*aQpM8>4txaz=_wXr3BRA|3jkq z^~`2R>7SBlD@k5#uL;=kB~)q2P4&%>HKYM1Z#sZ}0tDx$~2xl5A6FK3$Eco8n$} zG5=`~Pt;KdF}FLHfbtG76{Y)`tCBqtZ}S0nL$lK8Q`Xq$hA?x58bE5#IMN)z*KAXpWeAnTQWJJeSD<(eEnk!hOw`@knpqfs}U%nN5o5QSib5YxPJaH%A zVOTIVem>Wno@B@=ID}fdIhw<(Y2RsfV9_wp3SXjNI9hwSk5`_LLnt7Diw2E)pl~ox zccVo;43K470J~;-ZuM@eMI#;}#d6+}fG}SEp_LA8u0L5+PJkNIP>XK%a`4rbhIs4p zuRqyv|16rjjQ1)?OCsBYUfN!IeS`59_t!yP!T{eekh;jr@D+r9|6Y46wI`@#&SXd+ zc8bBb76L*7dVSA=-Vt*Zqp(++47VW-2%jN@YSbijX1i; zqv_V#Ueel_z)Yxnl)PFn)r$n?v%Un4q znutiCrbE7Li`<2ROk4%E>_dbGJWEr%`e@v|tlZ`r(Z+zX?bHfI=&kC_==qe_8Iys6 zUg6rRWHj=TO#K{XioS#&pvyH!S_E?OO})(6m9z>%-!^0n5_|A46uCOdzbEp=shdwK z>ApI%*VZ)i+q+>RMI7yZCn7~cI2L1hs({b(h!lC`%oAu`b3~8Wi&7p9dGu}qV>z2a zXRk;q7D$s3ZFm>Stn+GVZZYxt-`2SsO#fwfCi4(?XE7SOt8#X?4v6=dAfAGCP6YXs>mHrf9d}snv(j0=^c9|%*AqNFZ{X(9OXaW6b z9k(PNPCaUs(5ZqHGP)93w*4|iB&uz@0o2c+&v?IIxZoE4@vz&>`Bn7lP^v0#xQeVo z^C-S?V@LMdMrfV!tLP$gpS%D_Wpth!1u|A@sUc%9^QlsS7>VXiD;sc73CwkO^@b2| zJ;Wmc&HXPGG^s>;Co;HdDQlfWF?*s_{?TYv$gOr>bFF_e^;Uv}GP`7{Q$>O;;GXWw zbUUsCzKpq1JRa^)V|sv(-sqKS_Myi~FCog6f1nS=ZOxdrm5&oJb}{$Rt?&#ltm0+D zNyqOi&45bf>xQ%+S+EsQJJ_lMmP=|xKio$V0iBK%Sjq|j{b!m>eUcpVWMVQsdMzE6 z;iEdakf>4kre3H5{}nHxb|FMAbVfEtSi7i2)vUo>m)nNY5;nGmil2u)ewr11xx#Xf zj!uV536%J|XQ!wC3_2di?gZ5obBdATn|JGL#hWK8brnOwlugyTy40KU48^lO#)`}< z=2G0yn;>niEkaI=XV@U|Lu(rb{@-8nKDuDUNtdzANZZdvfXofCSV8UH@$C_>fTOxH(=edH!!`l;z_pCOSGTVL>MWzts8hRsG34h>!U zyz_c?RG((Z?!i|l0akVqhnMRx2x-qdqxN#a=}|eR_Ugk&o%_A)PXOXZ$(c_50-aw9 zn#KedK(0;m7G1LXz^kndiNPa*KIBzrp>^LJAu=bSRr$)Q!o;*-(bG8#gT=qeD5(R$ zWaG>I$_a5GO)sO3UG+==c(_!W^hVMTH5*DpOWXQ^MbLbAwR=TfRjIHX)fr*3W3)J! z?{X)!^gx1IFd>&)3hpFaj_}F_fvKj7b!d}6=coZe5-5$R&=JSW5La>3S;%iX!e}}? zLrRb&_Db*{<$h&F|3$1HiJh(gT?$MJF^EL(BAW(nK$y>NlKyYyb^5HIcf6=(g_`cp z^X5buO3-pB;>(!?sG#}*jlS2OOX@oGOz@i|qi9zeu;VWB^&lvP^mHz z#j@zXF3P!3lxv$SR}vF5N<+eH&uQG2U)qTmreeIaIkF{s=gsTak+U%~Ke|HMNE_!_7qq-Xn_YGbP`f-1L!Cn_t|2vkrjIE%^iI~qWGF3RHwJ*VENxd>^Jf9x7+H4sy`Xu#m=4oZrBpF4;O})bi!iqHRKQa`&mvQHe zZ3-%;r*6Ah0^R(bk*VT|;x5D)%Al?&R`o#65!AF54qKpjHOxUcu-skF)l7j8J zb0(=7@!7kDz>A``ia|K0F=}w>FO}tnTMQMBx<+~C%)L5rZ32)eYp2vAq9C_Wf&Fo4 zXlQSNaY-{;)+&T8&v@F6BZ(t(aiA+{W|V1K3M3XDSSVw1A^x^9SFHb%TYhSc+@iP6 zP0G3nl5h(`Gug$^`KpKtH1vAWRDV64VOEu|;?Whh{hh8EvKw+4v*+3)IfNft=s-F+ z)B82fZ;*k?sgI)|*O6Wiq$IeWRzXSrxKTdA^J&dbn(*MzbyrybeDnx;&3^G$o(~HB zN!MP4Hg<+x!)crzoJ|ONdG2B~sx#AAqZLZWpw4MVnc!JnPaN}b^)mYC>(6Y>rgIz& ziV;=mhhDC4qzFdGssi~0o~5_Bn7`FD+iV+TH{sSmELCmU2L}phT8#2$Jfhtuw(QuA zWYI4zI=qV_N+{tp{}hj}C6wA8dk;~*$tK-;xHc?>ChpknE>m_2zr{~(4)^Q4lVc59 zB9cCAMu@zjoc3};Fz_fweI&Lrb*%`iObm0KLEa0e@@VP(1nE-C(vS={XVK-^3`)`W zOpKp(iO|e(JUZmZgyJxMc;mj7_NPj%PEZ4Q&{Iv?f+IMXB|tS{!YU;m|2jQ}9_>Qdi7N7H@kePs0twd< z?{=M}o%pT)l%jHPtvZfLwj993QazHtV-jLkd@!FC;(ku$rLvdD%Ba#jEVnmHiXXRd z2k>wWR0KQo4N_WW_6sGgwOSX4Yt(lIhoq|yXK+mi6aCvFAF8)Y$wGZX_1TRJOn`89j3Bpp#i@l0?# z9RZy=-oTN&5y-zbJe~)NF9PI?fMjGA6d>DcuUk zbgC7$vfM}N$c!aPTCbax5fAq^>jH23=-pPQeUFAlI&0C$-+^p0dFd#%E~47k%($&w zJoKqemQ^}gNVq3B(6K#Ks^W=Kk8g6M(aPUpV5xbaHdCGVol2-^j{wc6SjD7$`0A1p zJD}+e$1x-0@D#&)I#uDrv-+O93Ex6z4pt`{jrNnfOcMS)UBG_DBrg$JqpYtC3s<-R z8YZ$+7Ek^yK#Zh99TbF{2f9&2e6+-|RqoC~TDQ%2e2vXEXP=;9Xet+@CqsgIb6(p! z_RUUfgWZ8s?PcuAyWt$hUrUMDEE~@HG)3r>F72iB@vI2{W!-8F6-{I2@D;(_a?DMh$Zald2v`M63SIyIdD+T`Vglv2GMu2q>K z3!T`Ph>s^yC}2Xr_-QOXZd%Hjbe#e^W4h@xKh-XPlvvI&>s8(}@6U#ot?PHTnwaALv4TKDlh>ZRW$h|hODgZD zMUdbsN!Oa1z?AWdjDV6LbrVSIdv_B~q)5p0mE$mpQ2fQplI=x4%fOQu#>+=Q=WIf)98QW z^Z|u|7b(l$g(Q9ZR~mbwkt0h~HjXjht?9}8Jg?E0@JeOKx&uqGjF{V2VFJG)>fk*3PP3n>oorN8L25F{`T4`-|Ht_M z^DplTtTF+^jZcRNHni#B8rfqnObWon*k*!?6Ex#1*0!zS9hnpiHNGs-XSZ6Wiqe*<04TR>XB4CFUD{ zVxMDI-ZSZ{x!TV&HUF&4aH|JK`|eWcP!Y-m_2d5^pcv6;<^BB|k}`Zuuio-8vQ(`^ z$8yKqGi!j<&epAa98+_U4Mq!gi^Z;&toDb%_pQW@#R zfM;ap26GrwB*9R}zTrl+%tbUxgWgJ14~zZK(F+@^MXaH+O7=TzWGQCh4Aa0(lI1Ty zoQ)@<*jam%xwkKBRkq_uv@sXj_t3&=gI|meKW@Lsom#i@nQ!IF{xf#F9u3&z`5yVp zoQ=6wBafHVaR%!_nU&d&McW?#lBU`tRE=s88X2-|By>cf^vS@OPUYiAOS{3*u?p+J zh2av;;eJi#QdCQHhTvFJI6OleZ}m9v&voM`Cf0*2dC%SK%x<2v5cZ_unPcI-4}Z)5 z6?-fYcnx=19}5!S6^vgRtvNW%v|onExk%%Y_(m{kb7pGBKcW(CXZA4qex{GDefgO& zqGaKL*>nHL8AjthkFZ;2n7o=h89am3#pT%wWI?+chNn-**B0y^8EuWmb=NGDoA&H_ zffOBjz^>8CxB_w5udw`(rB=W-=5}Nm#cdgnzFpOmZO0O^n-2M)R~gBe)Nm<+>FAK@ zC}Xduz!*(X2eCya!EXJnTgY?ILglK-Iq2is)$Ft#Z-zL&T8Fr__aY|?7?To65+O`- zDw0A~?am$m=GM_rV~TJ?ZP2x9&(8zqLh>TQR8q*}Gl=G@yj%sfg&?7GgCRU+8KH@; zJ+1So`gE&STNyGZwZ^jbS)-Bq!oj%mti>5KG6!vh8)zDe(oA(Ks_NgE7ur9Ksn*2Q zUeOzysWnXx)2+yfGuqgBwk^&W+J#6Z z=-iD*=hs0x;#Hgb8f;N2(W>?vH*6OM6wOPV7KfN!k$P1JyL%VcQ?6xp6gM?dvH=IY zISo3206Eo{0KKVK9Y?n{qzP(nT{Z1kkhK&dPY9cWK#&eck4$zZ48A2Ae;q(#($%Kv zIqs;GjHN%lHb_mw4nm;p7d~4L?Z5rtlOx*(hT()moQ$M^qX_>AA>x*$xu7QUw-f3erTp)myTNu@E z{wxCnfFVt?ncZivvENmpm^v>ETrC^ljluAfBJkpdn>5in zotWe{t&zbYo?-jB&xM0!uH54VvmF}qT){!)pERjR(CaQ{8IVt?#g*VksU0uP#LTwz zNZJbI$rje%kGS__&r~J}@*w(U+x-s2%`N+8m~aUqlMPg zL%>cXFEEi01|+~p_11ND5_sn?zUf~l?g|+w0AZSEP)Ub+RpZBbFiYESBQNR4Q#}4B zBV6<1oMsq^28H8CFt(@^13s$B+lhos&rndmh_A<`^+nTHMK;TB3y#cZ;v)uRYgX zbFEb>dits|TO>~F=tJ7=(GAi(P@kgYJP?Hcp(l9vp|Z z0|vQ5VLXmMxGm>&N50b+!?O4KzQ}^u7yWT`Eo#a>Q_8i&h_r@Q*5CUF?2gZ{WIe}&HrMoAQW2ek``t+5JYD29$J;? zTkO=;ZpCO>y3nW;lgsfhS^gYmVq{Ava-|tMI!68~sWAjZy8bk$g*MQ`l#U9Jb;(Bwh_89(&QgiH5VSZ6J~Fp_a>Q;pGRgv1f0h;t95;0fsI( zVX_TJgw_*g7N>al%|Ao;2)dRqJq_Jubars^%4YR4Q`KIQ?$kvy3a7W6_ViKbw@tx? zl|!ag!4_-hvrDpGnJkYSB#Uy)7hM09;|Ij03+u>It~C#| zrgo;0__SWT7;ErMD_8~wg7l9ntPSRmI!P;MO6I6bW0zx*$dSF5d9T125S3xaN68BiP}j}Onbur7KGeUVL7WaR(>)Bi64Q)TYje^#jnsKH5ZKO~py zZuW1getnM_)qB*mDaGzAUnRl8@zS$dCv&VT2g$0ePtnR^8J+%~l@uyKU&(ljogd?WW$gLQlsCAMXdDt6#KpYd{@XAjT| z$?hjGY>E0UKk)aQMDKYO3$1~I-gvapPT)>y(`Ng$Zo*KaT&mU%Q9Bt~zNB!Tw1;i+ z0eZKSIIbjzUMV?+YI|;?V;=oce?azMPTAu8n;~x2gFfrf345gnrZ6b#avC0V6#am| zj-BO&t-J^S4-)%_5`W;N0`HjtijJudU;hQte|f@hrJktqaclbZPLiag-LQictx^c! zoS6FSc43#4@)vPj!6Yh%x7McfK8s2@GPhIg~OmQD5Cx z!?o|azc(Q9l*9SRr<^$1R`wIDSv?&3qxJy%M=*oF>V*BYgil0~b@2$j>=s!ADxZs@ z-5YofG;*LK2Vy6Io!V4fXA=50Ym_T&pY4?A1G=u}4gKNwpc3@#d;f3Y^Cy6?78&*7 z;{5&_3c3GfJ}Z70$R!}W z4h{a_<#e1tH?y(IP2~rO7W)-XtW%@Cko!w7^S9rnQlyr)V$Xb(nrM1<~Izs*$@ zU2#iCWqL@hE6<)ABsYv!RZor1V||}+pr0#e(+hQ+yX|kmv241^kv0S>Kzw+jBF2U! z&_m{Q{BpWS64{gvmM}b#tlt@C_i^7#Qvx`Ce0wjIb!6r~fPs?vRfdO-E@#{F-I8N` zlMwa#4e1z1^)ZMlupFqG)6j@wdVYPqX`;I;J(vm5ZwuRhWyU^c62|MY!u`KZU1;HZ z9h6Tb&cki?zkWyrdNEAuKQE|_d{1rMTT_ZIpy*^bmT5K6GLzaHmPg;c)DffcF3j!j zMmci;;Xg{!5NoRbSs(`{{1KC|H`&+>#Y1a20jEH%{k}Vv!qE2hzQe3J*Eg%^k-}lZ zea;=A-iK5ojb-C)$;?nif4pWTGEAe9@*@(JUB*nKskrymcYsv1 z+x5OQnjYob*P%FCROj{^l&d|85eP|RUDu9lKNedL6T0K5fMU1~e;31b>{%jxu7Bgl zlB?l;Wm2Wy)TyXTCWhG7nLlYUP&jGPS+3|$tI=dZvN?3=p;~5>;C6Wm8r#;~)ysib z%oaK4mCF(~pKNq5q}E{8uCjo1j@SZe4ggN0mHYOQLbZGiDdBDpx88kca2LwhLEYAL z+GRZu^z;W{8znxMim#{2Ww;4Vezh*{O<_|H!{^iOcG&r;IaOsrrn7kYak<0(XzJq2 z=9bZDX7SAE6+U)`+0JDyRq|rr^H%LIfO3@4=tt|XJcNYuKt_k_9&EANTUKsv{>Z4P z-X{!$?e}4WvEb#i{V4q&&jUJC25Gx6qlttsEY6S3d!d5-uUpX)jlmSUOjV%lq;SRbv(pZxoZZ(ZWTqTyip zMP8F@y0D-Gy?x6mD*sOK9n^C|X`#0=F0$Ey|0DUafQUhHC|**#m>p8I}rfI?t072?|LSJda7w-W!2W|kJNj0 z^a)1v!J&l?wZ~RFU$I)>d6P>MZMMDqL7nZ47z%~(pTleA{fD&^-zoWGKc@nUa3%qc zqPG4f$6e;e&(T&1EaI%B{hIImUL??4^a|G0))X9b$<)fRgkW)w(U0udZmmp`tLJh% z?97B|bbDK$1RP$0&m81gHKMDh8*l0fT_r=2Uz^ZpA9gt)is=|@Jr zA-C@S>1r`*enfOT$Kbkfsx8;}=x$IsNcZkL4~8X$_?nrO)MiL6Q{0CmA8FZaD~8#D z)?xW)=}492lnAi+;R7j5;X+(yum2fn;KVCV@)HC&`aquBHS=zqB(5$$^+xd3cEq7y zzy14&*nvytw%2ejpb@lef-*R7e7{P^Ok>NZ8A7G}+3Ms1d3BZJB~rAb@xI4ws3zcz ze=NeGT`$9h(<7~|^3L@fIu`+6Y7>XslhV%h4WVch zxCa}90^ZNs43{SYvnH~@>`hl!w!JBpBkaO};LyIzIBbdL6*_w8A*#soMwzFz#B7>V zj~Lt(V@^-LE20v9T*FB+WA;Fv;2hhD{WpDEf1u8LJgZ*hQTGe4fV`zs%;`dDu&O+j z?6BvxDDKtzqvYh|=;l|;PbRrk*^7L9@%GSJoZ;LC^JD>_C2gY zo3tZ^(?2(OlDMR`?6$8{@hP>EwQ03K^UQ_F1zx8nAg@h7WQYd6vRmGPu|_+Vs-FILAMT%&a`O_!LYFtJ#s$gBiN0cDzpIv(zrW zG*%oh(^Nac6DOAms)x6AS^5YEG=_gypkVyf=oTK&&v7u)nKNYX87X%UkNAWR^4V)Q zD^bxGg(2AG3mu``9mZ)KwkRld%H3g2Rhw~eg@@>vm~6cXwQv-Q7Hj>nWTYw_(O8lCf$-K7RPW>`ptiCT{yDoRI(4dHG z^&)3IHOZ(cM9~nEKpz;^pd7IRsu>-INi(&|+;HeGQLhfFzgWKZ-(D_=F(uqMBE6X< z*Dm>-xR1kbdlr3`O|4mgJ5>>mO>ohrfvz+L1@vSRrX$q_i5`BOGr)A}L+Ws29$^=N zg(WM}EPpmvJiYwx@>LkH8S${nzTumxF%Zc- z*p%9Ni4ce+eAd{lX{W=3|Me0SXx5Tdu%IT=D}p-Q9wpGx$+$4@p|h?VDV}}1PMpGI z>5^=li?B30F<4!Ab`Xt_>r^j9rPC?%nP8nv5+}S`-3-UCOd3+59|mV}_kAyju5o-R4KCT+|7@^t>2RWM7j;pUn@@-t?XWZmWsg!R@5R1@WCi9jT@qXR zclFTFx!hLa4ySc7_C>20t|1v9owwKre9_L;H6X9B${%9`wD>V+3Oy1%-CZiSO#_0H zI4lLDL+iy2mF^(#qmUaU0>qm~jAuzPe2KozyKEagK>Y;7EI-ep-Mi!ccfINd6wr!N zvX{}wlU zdQ)(Y3+WmKhJ`Juq<>&p?`JVYRgiO3*|&z4A{j);4E9&z@zzv!r8 zPNvojn2+iEo5}D=X8AQWwfR;9f#XTcrfWs97QyE4BOZ-&KBF!0RbS#%{}d^hR5SM8 zPKRa&V|)r44JUEW&yVQj&^wCN&XY(J<(s3#4=mHDb&sff^H6+a_qymN2QCZ1PjAQj z_cQIfzEP(A@$lX^3aS#G>g9>)QWBo zj%lkdRPLy?0XF7J`YQ22Rvgxwm+Rn+@1!YI29nz8V6pucz7NqB7?!dkBJ!5iBXzUP zGle=}H??m%E^}(LV;oY?lRvd|QP1t4nsgxIr=jf^|GR87{W-{1RR_!Hm`^!7Q~@Z&*iq&pU?`gYh(IKUr6wSv^Bt-P-!h198{{#eyV~Z+geGe%o_agx%k?pv%KRKc=^f$lS}bpDx55AkVx5dyeO-IynUEnWmW0MxYdR9 z_BJ2{n`x6fFLyk$&ut!wcVwx!f`A93TuV#qiA<@K#NMbi0aEAKZqxEH9j2*{+?s=k zbAUnLQQLD`pp0#R48Nh9{%CbiC8)ru5D|M$`H-);U8ofcIU<;XXj%>VP?lR836J*^ z!?NGj_>Vz?B_`4$((Hg{8dgB3o5l-;be?QrkQuXX=mWo-NV**b>5XMcI>HE3Diuvx z9+{mTW*6qiFzh8WA&JA)XFKpLYX*{qyf*&#I zZE5WKvbl{A;)N;p&xQwl0{N58kt;tl^;9XYAAr|@W2=6F#h%SZtN^F^Z1zh`3!^B==Ch8`YwC*>XShtxz&k81mv-zn2WV|#+<0ZTb7vRa*LFwVj2>~ z9%)E|{%4vM^rt{83UECip;S&Y3RR+v+o3t%Z6 z7pkJ3RO}x<6nl+VWFCNNp33d4*raq|JdC{Ud4&&j;*{st@j+CqcOB3sSF5Nwd&FbE z#ST7N>$FB+8cY3VuuEBCc8EB)&+?M0QOoZcDR+`DWl9n3`3JnQcUfVS^ z;qX`Ct5m(d`xDFj{1)49o@G#F*ddU*yRdy!otNHpp!98 z?F}LhSBU9)7|*GI`*X|$SJWzbVr>&rtM2Wj(lHzmq#Pxy?RkYqgRmMFjY>NG(aP!2 zx}oG<@5Fvf`|$xJ{W+`IH8KJY^WdBX622gbRmaC!kjh}>#=^EH68D@EeyZ(y=ZakM zg)pi4^1}B9cAFCdw=y>dH!eHhl>u-m%)+Rtk_nW!PpA2G^#b1hnNjK9`?mR2?e>6( zd5>xolvm0yx)d21jtqKZXqs$b9=X)daxc>S99lsE-#qi^%YjKXN_W_Q{V}m=vEtQ5 zwn<*F|GrN+NM~mwUX+bEI%r}uwY4?7+*uK9KlEX@k|EnH!^ZY*;SI`se4vhZXS%?8Jbg#5skuj#Npq`6jWfg#UK=dFs$LwrrD(c%mV%`-Zp4hN zWTEy9mfBf_BIr%alhx?=|M>fa9LNiQfdlO+X}6$|gl|c=6omc0<+bb0DO>}w$K@Ic zSah=9hxUAKAb#0CCFv$4MQOfq;cZrgQwqA$&z#jmKap=jI7AWjB3nT9{#;lhyc9(% z!~0&QH+I7BGhoL#D2g)8&zmc&LgnQ)+AF2t3**`MzE5&gj&l2#?Z0hVaj9`#(51Fy zb2V$LGVUoOg`l*YUiFz;`+-CkKZRg3F0B>g7_9G<5j`%R#M~dQ5c^nVXI}oi!c6 z9H)>l+_i8jW$!Y(N&eGpXC`LLDY;b_RexAY!T7A!C}vQ8vYyS1U;(H`^|xh8n2!>1 zPz|UX8IaY{V#>}YzMiV(-cV0gzmW*4HZYSNnZhlejJO@^2RiuiYYoWxWUVAufC7M$ zlB+|j!DomJXVq;#t&XND3;niiFQY>()6QWtP1jQ?MJy`1x zkiS;Mm9t{+v1Obm&RD)Fy;*yXDct^?S#e82h0~S@@Q6R-d75|Du_P&MfT}B1C^Om^ z3(2x9^FTeMGbPcsnEjgr6PC0V7FPD}G3z-7aSi*DSq6J!P~BOfHMR|x<5YxgQ&4-v zcp%)hV*~y4Whx75WM(<{h?V*EYaMTnAQ<2-UcSsYZJ0|@^ia0UwzdAwMu}NO3UwPJ z5r#?n5PtkR7p;KNI{$T>1IZj-0Y5~UEyv(t5qAX&tEUNtg3vBp$J1(DPANGZHvDdH z{B#DFIE;Wi%WeuZlhaMdOK)}EyDS`m%{NnN`t@Y}l~p*ZnYhjNQqeuJ>F;feQGIA+ z5tr1+y7iZ{ffa-D_U%z7tqL<06bZ*4WA|%06`?w%0N%F-7CNc8IBKW|3VJd?qaoZ0T2 z{0#s-1Vg`{PrUuql7|p6;rqY4Awp}^bo#`a@$ho8|Ce!p@Tad{k9=iu2+Mv4O%ZR0 zNFvCTwVYg5BwW}!xPBu`e|_312iSPsaxRt|YUII&&XK?0-lIJZ?!htvWkEitPtDk< zEMJgd!vqg?Ntb`B)|o+7S=`~R_9(S+&Fy}nGPJ|Y z2&SRS<|%q}GnYRHt8s$RD8i!1!u8s__1Qg^_{>2^Nu!Ii-LRA2^gHJoJoP5yf~ot< zfYMKS^7V~}Q*zkM7w$7)`^PzA(Xz+eAv3i{3kwkY>K?b4fy2-kFfu*YJDzHF-thYuRnkP6H_`I)}uyqzQEn* z;4mRu(3WgnBPx_AAdvWlv4pV{OQdk)00mS!!&0Wdh&$7V;WKEZIXE~d>COt))YN^L zzn%R7D&f&zk{HhDsAIv|oE_2|4LXIpaH14LqkH1$h_sC5y2kFj^nxbHV)hE>Xq_s6 zf#h)oUKyY620Q5$M?T~lmBn;Ks0oReS*x{z_9QqLI!pO;-%2L zwY^Q|@qFdM1Ic*5T>XXE_c1$buW7mqA1Pdc0v}|{7#z>JC z=DiCt9zHS$)bb$^`O?xpSf`KEZil%wYHoCr+LuB`S7+}bjpV*R%Xq!PenC`BaS(YrD42xs7S^~_?fF{9mn z{!BAszL4;YU?P}KiF%0DC4c|>Y(0_M#$2T<8BoSF#Jjt7;=nz6KQ-WFzF1wq2Hhj_ zcQXpk1Iq#(TsQ4v2I0+z#&bJrC zj(*}Fdd9RF(&uG%iNN*{41Ex^$NUi=O^88Hq9%c1f8>*lMaR)(Op?kad(w+Y-Q_G8 zJIr29`p1>gM%21>R&#)O@koXylOH*5v?VWK0GMU%HQxjo%EO(X}XT=9=-fLa-o(-nOV_Ms6$)g!Q`HK+letr6Zz|`E${(N zU1CPx@Tzl<_nO+bW6qm~%OJDEBJE~Dc4nKN7f-V?YsuZS!eHy=L=*(mbY0)>1B-E( zMat6k|0ISFTy=zTsx9XIyixF$sUp)rdELI0LSWW*ZM*9^P}U z?ZNBxCKsJkw|sZ_ZQv(q#8%z7Hx!r3^K$qtgM)+Yj*CTyj6f+(naq8>m6Kj`vvLLJ z`h@DWORaq=H8*@N6X>}j1)=9zLw6M@0T9HEe1F~+61%+#fmJ7ak=858iXUL%z>evGoL8SyybEjKL1&8;m9(l195R z*oTzlIh!XWS;l2#Wf@JU{BHH}u2 z#9?`So;A~TT#M3w)2YIE{sofaj|Cvs8d{aCsA*h!82f6}G%K{abdO{)AX6b==HP&y zCP-{KlhpRKClMJST1mw~0*m}YWfpw~voruG&(kbSg;%YYCI}{{Tn*|b+g@48+b)|I zY>WBY{Eaq4$1xNnbWpXzW44n{3G^f%^Y=UGL`0NsgtY&L=s4bgDcC|jf72a(<9_H|F%UNoW<`LcBJI3JH8#E3J$ zaa^xGX2?v=e`m@n0Z}QKW)yigI}OPEc8hD|eM@;<&zOIqjqxIHE=T?dHfi2_uHAp6 zFOdZntX>%$1*^z3%&?KQW{4j&I5T`4r@J_t7i;#AY)qv`PK)8bD65JQ4`S_TkM?HgZxm%D4G?Sl9X%%Xs?5y*~{BrDE}9SY1yc$ z!|f>Kyrbln_3iM&I0i`WT-Y@z1_BRydW{30DbY&d8breJWLpVLQ2YBSmg6I@^BMiE zZ(9!!bR?SCQ}EdYMMRdeyJyva2!2hzOkE^W+#le2;qouba%${TJ0Bt*0O_ld&=Ly1 z%<3%FeF}c_n;GIJf$BovYUB>qq*Ff%<~3JC*;3?B7HbrrT)+x~^J2>FZ!ex^WB7|> zY^lu^T}*GuTjK{RER4z+JfT(itn~B+_8ca1_ptO82M34V^x3?99Mp~9CO8wtgjK?s zS6Fz?Ut=6(Zg`4KvZq$AQ{$I2aIt1&a+lbBs9G)c_{9d7gtJ^-x_~RPCBfoA%Cx=8 zw1c_3Y^jvM$L#lr@`C(YJ?zh z_=}3oyyJ3alZh8VGeS;<-8~*pqRJ&A{B!`R`zX&{v%9WE=S-JPRRP2zUd#E7)?=dc z>#nv0ED-;2hjK}wRg(}r=5R6{1yZkt=DYwc=}3q2J)t=Ktx*mUf6kwa(#%4=9Ta~= z9&6RUL`p^E%a#W&BK8oTX)2vy8Io|DvL9PsG4CO`8hz=Z2F@)nPuS zvwEvT?rYwS&j>i4f~`*_=?%wFR%2d;He*NGr|AlbXL?H;0?npg5es%0wi+oYN~{yi z;|guC0HmH8dP${q*4mYpU@ydCmrZSFrl4)w4so(!SB~0YsMu7; zcW+ngl?}cN-AmmIhYk2H5f|-e&4uGdBf3Y}q;?k5jZgNSszqGwrHOB8?@aPj&#LE( zeSA95E#-fs^%fm+uy_%6{iKog?%2d&6Us zgBZdTK1Hym?RtN(zBaXB(=88!dAFGaVtj_L6gJVh%kdckkJIv6+F-@uw94(JV@)Qv z4}v}ad&6R&{PZHp?C*WAZPg%1rDa9+yv zOPjOeah!AGllvo{qW-dkb1VlS>tu+&rLKGpv`_SRcxiQRCGvBXkW2WhsV-@ArBo4> z;XJsu|Ljn#x1n|{(D`&HiEDg{S8FH53VLaiA~mfF?Xc#=>ilOdi6mrNuhlNO*k1gs z{9|{}Hd|>s15xT%Y^IMRM4pn;Q+fg&s^xTZAmf-HW}8#F$P%i8DB*2$tHz zrS6hbYk$sxo{Pm%2OqBw=#Flovs+n}?kVTmIfLg-A8BZ6T61im0QhO~Y&XeaGNM06 zJl0#fUTUJs{kg}~*D?0FF^;vq6eK+N`V7%8OYF`^>`Fx%e#>OH!w!^9s(5q7ZV@!< z4m-0DwpjPtA4|Ag4jW;dmD@O_G%@eI=R5n<*<=a?v00k>f22?$A>vQ@lZs<505O9) zOB_pWX7d@`FewV_jh`M8sauu?S+d@_Mq@Rk9Tcpw7Z}tsSj)oFZTOC6$GCHJ9S%>O#I$NBzfeabU*>j+EQwPIY zuk}$Iwi?=8-6@+wsS=K!@YCjMajJkDWP#2_z!Tr%y%U_BLWtmXf9l5U;d998CkAA7 zFK#Ubd4}V%G7~XUij8-Mz`u2Js`;WAOFc>d(McOD+lazR@lL5`zQ!JPv|y~Q0YU(9 zZuxnNS+rN@c<^MPcp6DS7a8IpN#(fjw*bT&x^JT@8F>ac|6EvLO6VC&;Akwc)IMt1 z6Gs&u&X)2|<++xrk+Hr{hq3LTV^h&0b4ZsSibW^ySV0Y^ElpCRgDEfdY59Klu&& zUfJ8F#r8FfR%V6ZuJ>vAiJ=!3*vcbgV+Y!AkCR*YSH_zejI6wMnr-q#k{v{{GX(-& zm5wG}(7qMt+hE_crN!2G+c&K$e?~{VFr|KJ0<|yVLNEga7692#0+W7wOKn;!ha#m$ zjTQd8cX6kJC-Mh+WQ(aT@E!w=nt26jpVNv{KkvUZ%ryJmSee3%VUqh38|#9Ji%$b* zJx;X96VqmZFi(G(6B%#uN>>bK?XgoneSMVOILGm0B07P?dlg~yy6%6ceN&T)HsL!c& z{9B{P^L&sHp9Si~Fz8{-)L3gyM^yv#8D;6z)|FGI_B0_KveGiuQoeeL*^`=ltG4jy zN1TRdy4jLZh;g*3A^?n)0UKFo9lT1zO9i@NJscg;)W5^(sGN*|0yA*G={cPc)-J%x z$rZ?a;Kftl0<|N=16czFsh=I&z9K(9A#em_*vg(e#)yE2%v)*xsqLYB&kKa|RT&N4 z`x&;LIk88>6$v#pT9f%DsTAfjI;vXH_1+}>^<-mjCl!a2-2xtNeeVQTb2UM}Gn;6S z@8QQ$x%Tf3IeUxnr~M|ooDj_t^gi7&N7E`Qsb88MYA}_KqHDXA^6(2S;1=09i%r&h z#=$?Ekr`i=M=A&>2Eq!LHnZ8 zR5<8Q(*7y*vq`QuA43*h%J^yl#*yjF@pI>ZaP#7%-!yBTkBiOq*60TqH|8huZ~FKb zKMS3{MUs{#{lv@6d+4#SfIpkcWr#ohq8#%#+XL2xUJ{~cS(oPPbdpMx0tp@tQ6jCq z+?i@X1p;s%T)KhW>VEX{Tsj%89LGfhda*QAK%~}5)>Hzrkt&|fQh-wxZ;wO}ztOFp zj{c(GwmTpx3CyX80|4kT~sG#zezZyu(+F7#{`6&J8i5)@BhvPH&KcpbxgGtc6p z`?PQ-MM_HQM>IqEp7NgcUO%9_Z7 zdvm^36Yj4Ez>&ktILZ<8DZ7Z;Sy}u{)$)&{?yEHSimreVdVKYQqAQ2mGX-?ZQKD8z zqCJQ85mwuC#iKB>FIG)9bUGQ1eOHJ7;r7GGIR z^a8Q&pwid^vR09dPFIFY0d??hC@dxOedwMd+eRuMA}p-p0k4(z$4}HoOjyhR9BwKfe<38_)_VI)@&y5lC=Xh^aFbv}PkvhsMjseK zlYz>LXl91d2$_!G_h(F_lvMvjt{^|f<12(=XCcxNv-qPQAtB(>cAm7Vlu^SF%zPG{ zj$oC%Ha3x3I`11(%1*;sP6Yb&efnL}sSJ0_6x5S{NRF7A$(IR?AC-t~3Jd*-N=>Yl zTZV%Wkj*Wr5CH*I``7P!c0XpJx`UNk_JS~X?>wf{+ux_L;t{#}3nL_9H*h-cUp`+_ zo$8pzd_Z#e)|V9<_njBoi6r>@%;2$qs7dV{QXwdB9lJVNM5KC+aYbeRy>!`HtOjWy3uQ3Xny!n^QAR}_{RMLt$f>_?Tgnm8qA0_`EjgsEI zM#*UTca73_AbcOKY{Cp}C^6<9giaHP$eUulH)Blmkj`)iNR|bSf8eeu;tl;?DG! zedRv(Mr+;#HoJ7Kr9O7toB1~P-H!>CH#PS-$b;Df2FQJf+gIBCnPmu@Ei;#y)^~>j zLE350Y`0zDvtX11z4xx#O`84?qltn8<%r}6qlqYHHoWGx`?}e_3RQ)8ufI~NE!c2< zq;|{!T0DEUwx7;%v29o$AFCoKpfkt69+p4&JO2lwA`@wE{)(&AVMoFO2oIY`>8q;s&P$&oo_=qY*ohF!<01rk|^7Kh3 z*OBuM+j7Gr={Gpg*N)WA)GXs1&PC-jq~nREDQjObZ^sK8^yA zV~s?mmi06cNz-yql+H1=@b#aK9~P+v`9eB;4!xY}RZXw=#}VKfUuECDBw8%*j-~~r zea1ZBsy*g)M^w&!_8UlWa(^~K8Ptszpsa5ut1z`&Ms%}F6T731`lel-sfwI(yu8!z zq}qUwn|ox71KZH@%cUGzW!x*sLmUj)?$=L$iH+6(u@S%{R9cD*nR_l~*gcu8eQsEZ zX*5>EVYQohiP~zrCfNyoQVt|dg^8D?ewMDAa5ntIEd*Yky^H{VMG3nE@(Ex(I0xC8 zqgTvj!Ycu38A(#e# z(7}L>B^*R{0)(hfDa(Xp1t0>el47zWZ~vRDBp1d&N;`e{D-a)r1-i5{^Y~Ik8MaVa zdhE=evklePe)rrVi_pa_OHg$Esi zv=^xNMmiN-XMUdAbh2KCdbhZkB?7pwblluLF^{eQnWfE9^YUS>?b@LGrSPA;&}-mJ ze(Et_7IW;!|M_5%Fi>o9FRpk?W@}8qX|RKXqA8pR$#R^;VxtJ87hC9>lIxNEKYs(5 zf8kfJkHP@!!l-UFNBbwpc#RJIba9w8rSLEo7Z>*>I=U|!#Ul|#w*SdEAs+hwem$Tv z&QP@+fUa@5JYWD@Xi;uFj7n^ChqZlWGSFT@^9mJ>CR9jQ~J!USutn78- za#2G0o5Wh1%x7ePG>g5yjWyCVWkQL1@px5@ELl_c)g|?|8yoYB3?ibyO$QETK(jEO z43ItJZeHv>zpcPuki7?{rDbpF_QuudLP>9DrWOIHeL92(L=&OOL`3X%ej)%>Z(0E> z_A}i>&&SuFnDMAlNbd-i?RKz_I8ksI%Z`KmUeYGX4s19rlcZH&G`8ESbf2ZsjdL#U zpbU$a*y^g$^vA$NvU-tO`$nF*AMO3I4+1YyIJ^K6_`+b8h=C)6s|6wB#~(hEOqdyF z2}cuOH(@pEF+uSB(I9jY})rX}2|6Diq{<_@bDUOoA&%}fH zPvy2Icm|4KM1&mS%Bde|r7s!tucLk8F<6-S;-t%xXp&sPRT@LJnF@i77!Ys(hbmZx z*c5GXSOJYiLVJol=#|ky)bCSxK_6wo<5+*FYFF(9t1=YT^&t`VS`QkKM=6%HlmvJj zJn~|CV}e3}h#R=nF4mFp&2AFfZ1$F`JFYmu5j;(z+&{za2UIwRJ+R^73C+}Qa zdA|91h_q+v`m$W5(&Y&v?o8osEQKci{`sp%%F4#n%^QD~w$TP+RDq6E?sZ)ACZ3<7SwlISF3DPuN>5qX0BHpc$InS+b)>vk%L#yc$ zSuGGnK+)`WMtpCiPua=`^WMobn*!uE8jiSIAE!Edo-&B&>J+;OrqaL0(eiWS?YH*% z=B}=1O_q}{vPHdsbh1qQX+I2MvLcFRPxvrAzM!&Of#O$r@`b!EjS#5BsytMXbX-4A z8YFp1!w2Dy4S>$~SCrJI=~B*PBm^^q-bC>G@3Mh5QhM7f{*`0Obm?S1$Bv1`uP9ss z0s^6(wYICm6r>hf4K70KjJRwD#I)oY_c|&W^*{~$z0?i9ytuNa3`YASGj6%N=|*=d zfYw9E5(!Zz)*xZ$;#wxrC%M*QKii!TrHnEC(A}2cG(Ip+KXPH*C+t#vqQbr}o-Mwf z2}$L)pJ5jUlbEM0uLLqxkb~{FE%QpaDLO-5e6oyFdIp!>w%x=I{E_p28d zM%Qr5_Ak)8ZZ`h^FkJ;tBp4!D?O$bSx>C|4x-PHvZEwsp6yuLI1gBL^c7XEh#&}-r z@Xg@c{+5aRYw59~0tf<7Tdb8n=i(&;$l5ei(Zq*~Of>NUpH=@|GRc#bD;2V-t5J*M zlhqkcPB-A?lnxemJ>Rz2k#Eu>(VLZ_P|k(ch*n|$sG^O_afNg{aYps#iNyF^EV8yVkWmddSn+GKZRf$UcI8;-SpH4|Kt7{uMW z31*Q)jL@KywG5?%_kYHuW(;Hhm9c_BgO^o}LZac1Q4%bjLaiXp8W-U!CrF z8#BG3fZ`Ag&I3uM$Y@1A)Jwfays>>79LZwZ=JdWwAyYpZMH=BBLhJAkLW_Cq9YABn zbeb8Wk;{m>g8S7k@{eA+f5M-2Apsq2G&u5U%tr;x+HZk#1`w1#?}sF$wS9v@qfmGO zl)iZfWYv`Et+To{?cwO9nJAiMSZ5iy^fOkw!#dy$1a_Kj%ITL~daljmJS!Z5G&KH5 z;i9`2L+y`yc>nxb-(g)mXAx{NYO(|fq%RbnwJa`5w>b%rA=kQ3JHGQM{XEryNrize zb@->1V)=^~&efLQW0N!n*Kqm{Vs^&HK19)4C|*aiV}zPjEe@JAJ6jApnnPFldTm^X z#-!3E;2$ZS;=Yq9+4cPi1aQvSK_K`z92>SaE2C?)#a?EG0<+J6#Lb7~4sUEOlDH&@ zoKmP|JBXfgoU!7)s~cC z=t@$(S3}9ArpVdkyVB+eN}uEIv0F0=x)$e(#XJpOEB&eCgh`VCYTZbcX(^v6_rXJy z*UCcSWqFps8Rj>GzA)cl zgLnK_jr$4;Fx7?{30t-NDDvf#%^hPm4bcPuj3_6XBb%&!7 zh6;;GjMZ+X=1)jzP8g0x^XST-=_tY%w|*qC5zKARcF`+%1{oeECj@^{D~s{icY=^Z zXyfZY_^I;!$?m@n0Zg9skZb&&YVAFVA~7*AQizZ4SpEO7_m)vnwqN+DVu7d#f{3Jq zlz@Orhje$RlG5F6(cLXAjdX+3Ees6JDBVcI0K>V*x4_rm|6S*-v(BgU!L@h>dFFZU z*!SMozV@|Ga}1kT&iUG&iI-P#1Y2DBT2h8V}!7p86^aE_$nQd%_ zV@7c=!waMhbriLuZycg078=)dpbF2L^tk`I7-P zw_HX`C0kp1{rk#$En?diz-}t_995q)lZ=O6L67*Ln&xblI6f3oqcE?S_X;h5fSD>v zp9({sr|uz>1vNsKlD}46@E%U#*i%s8KG27|Xan=8!89AUZ>E&z5h|k@iez=2pmHoY zWN$T5sP&-8TJGa1gb+1>5=Y$SDS?e%QkdX@G(w` z^Xf-5#}YZ5>uH%GD(A@7UWOzgA$7XQQ6@zeC|ZLwZ5G+fHIU+J4NN=dJMrGR%^-%! z-Jn^=bPE%64&)Q`Qt+lBanFEy_LSZ`~N=A-KeuE$o#wzjLw zghxyUW?>w*6N|o7O6ix^1t*$-0>I0S`pJXcioh{+o9_0_SCD!6l|8FK0Sh6C8V!#9#Vsk{7@-neTqHNi6x<2nR6)Lcbw# z(E}(OGw$bqXi-X49Ek}DA=J7Cgx2K~GZTpmy=|zjnOE~iAFt_Vn9j zyyme18r+-*EV*wFa5fnq?G&W=_arM9YI*sugg>ryQaI3~P7(XMKD2!ayImWVEFxu<|xZPJ+XGs=nKjz$zpXbz?#U!S554AOl zKjRU9%eWn)k)lo+S+t@r?%tmzsl`_(HX_$yclhIbcu@uE<=z-}ToAD||B!uCy6|C^ z2-Au_3u|8OpR`EIS3aFw%aa4blOF74+}WjvK)&TeXZ(;%`3`;@f3+=r zwI=6l!Sp(-?S~qglL`oERJA4)CIjEcLy@K+^7p)Nby~0LuDVn*7kp}j|79(rTIvDC z2Y4xSAHQ)?%h@yxGDns(`-fh!t0L0XBdJwpC&+UPVQN)>fBk91D z(TRgMs4h)>%DRiVOCP5Tos#l(C0*pv`xyrPj)&VY2ELugmAX5>2$&ny7-@e!#?yFO-&} z!+v@&PsLX_TY`Ngr(I+I^gfGm!G>US)dti(t)(|#tk!JgL-v*}Vfq(@-vwv<^QDmN z9`k1*MiC7FG2RXDqe`)fzs`RY{1a)N5uwEp%3HZ>l^VtCtv4&Rbit?hH~ z+vcewo#H_m5mZ`o9)6zcWyX>~wMC|98?zn`N1r6xzRR4o#(|KT-Vzc?8Ue+@EAF&P z?&br*XBo|47QKd(JmGFUdsALS4N&HY9@nDqLn_5dECzVBz~?dZe6vZ!sHVo)s;??8;X#x+2}S}Ox%AfT;e4v8ITRP~}6Mu&u+Wluf!GkDLq zH}W{((*h*y>v8q^u~v?jv|0eka+GwYw^? z&l`OM1Xu%6+-^b|H?IddQ0>QWPJb8WK>Yz@)yb@I@*lnm4>&Lis;lZeuw?L~=a*0bkjt=kvX1!b#h-!49N z#EO$@Bf`&|G0j&mwvdd$w&($&Z8M`;@J&3^9(PAas@dH~g7XKQ0bvaGR`|^o9H|M{Qncu%e$YoSe>xSaruKsVB=eM8uB!d?|IQ;SG{2_nlk2?#bWo0K|fT=0x{7y6Q0 za)xLvzkSo1ER-OMLi4h8PAu>hc6zkUwbu{Dm4-QkWQ0*Utp;7X@9jDe>gNt`4V@MpQ=sfOHl8X zG!@JB%Tm%(2kal<&Tm8m^%)o6*nNk`Jg%RFo8AWqu1`40l6X9Kdthj2==#Jy49*{n zzPCF5&QTn;p~b3wvI4%F_1`rF7X)6<;c=NEU6a#9GCbQWMW_V=})i&b{V zQN&{{6p--0m$(POs0j^?p+iC-YI0xb2BFD4<2^P2L`Bfa6LHv^3m0fOF*-Lpx@nyj z<@ZljuRvsS zQlsm1w_h+5UJbs-ehaw$|5daHD$xyZ&{CA%!sAq{X!wI(^=xNdGlAO;3lOXXU~PLQ z*(Uj#C;sm3On(>comcz-YrN4!`VSqrVT4KC?8AcrP9cf_1t6Aq8xzmy?Z^i$`F~0L zJ*j;2hKz{Y^~Gx2c~lQDvnZtr+XnQ}`^POjN3oxoAx2jn*R%Ys=EQCTvBcC`&TG?N zTl8y_g*Bg~K}s~hjz2>q_PV94&i9a)Za+EKy5og0*pD?`;+-8Gb6}A_>-A1*inaUP z!Lt5;T&{=7)O8+xO2cx3eI4$I>LCP$fG@@ZL{V?v1afg%3bQfi%k_}((*-R3OoPCC z3|+tSsQ~Bvja&*WH`Nckb#y4c8>h_@}L}^<4U}Zfj^`JcZr?T zH~*050P2CCdPZW;S8x{bH8yOb%=X6gb*gstELbGA#w5c^TnEza0ITnza8#~SL{^p~ z-BlV)na+emFS!2VTK1Q@_Y*f63GVak6ONeOH7*lL(GhcXK>}0n9g(apjN4+NKdMOs zI?HaD4G<*TEaf1EAN#pm>bWCB%(pm=d)<>KgzWH;QSXfOox~-2Y zt!ApN#teHC-un2-2~0%%RDbJe!eiCHWuASz`Fl#rJ;qfsiP*az1Q(a&7mTHWlD%~H z%Fze&$*YS~C3Vjz`OGKTExtp0)RCwZL8MPLQXB4t*-ve_l)y9Ea62(ekojzHPD0DQ z2_p(u=wqo|mG4*|RZ@yKEAy(`j1^5t8ClQq0exI>QzGwF7*LY9|OfKH`w;Y*glO@&-;NnonV^1q$H<~m+j5WX12(p|Y zk}Q#6REulm?-%rp=vr=(uG~NsD6RYDR?=iI3aM!JfDY3b=dtsyu!;(m`}mf40==CH z%h%tL#i$!-g@t*1Q?ZU&d5p#!-IasPng-on?f$9MGeDwLi(J$pE%}z>L0fqX}`CjfksK-&L9C8 zc7P^E`7jXk5GU@$#JF?cL>P0}nFS zbFyaT(`7u`L!xOg28o>shVBf#59?&hC=e3jM5{3QRhZ9_5X}cy-wKCjcYKI;Mnbyz z8RoS8+LSM-?DMgi`ugo3YT9no+W~rHXLEEWn79xu@04 zRjb&5xtAkT4g}O>T?~Wp*oe%=Sep28m{|424AbZg6_v+KYBPj($7rz40!XMImpT3KlapW8O|Uw+SeB$0G612(RPP z_}Q80@J)0gGVSx7$AsRFwfRudP>|1-BT5$}IsGZ}iawV+Cf}u1LwTy`u3puP9YY@}p!fM{66Gvi;LGt9eiERfSJoz`6^V3GNwMb7`A@ksXD2SS&;%Zji#R&Z`1ULI%9qFI zLZVYdL?&c)dESO&DY}gv&|sEu1BB$qA+En|^hDM6H=nKc2syy3k0jgsSfK74o@LuY zyZ80`rGxo#oK||}2o{U9t35qm=I%@9bdv4zU_R5eVO%5I^%lF)kiYKKS-ROOpCZ=% zbfP3nm_F@?@oM9FwE!*jrcm0eAvK?aE8xAZB zkQN{J+wZtfTu4l!Ru@N7@ZtOT?*;uAK0H5Zr74u5B30xj6cQ%F^ zBPObP7B#8T1W*%Fwv@%Ry1u7fT?)w^aMylkxV!VC8Gnuj3DRVsm7w5$>7l!H_101# zDh{9WmlC1ucAD|>(}I@fi|=8PiXTUdemn-QkA`wB$*^mK7_oW@CZQP@mbj~pMC?Gv ztG`MQBZ;3pHORIsB^>j2JOWM6?J3Bkpx<@g6l%0`&uyw^SlfZrha!4Hn8_Td&@2;i zQn&i3T1Q`;4Li-c93%|GxN3qwEf`NJtu<5_iABeub0Q5(1FlHO<8oYvq$W^-nDVNB zdhF6pM5o|#BnwYa?&I=3Y}nppf6?`CZ6e29(D*S-wSa!t^Ib`N+CK%2A;JGGKr)CC{dZ;`WD%VtxNFuxJmT=KYzDH6F zn*Vs)<)^F>tRc^F-o0%+9j(Fm@zJlqYsQ=WEiV=y-_4XauXi5$`T0DdP91QOKWx65 zN|H)NiYgLGBLUd+jajH&#y?etn zplLMrImyE1MT@bGk^vZYN<*v-k71Q+SlgX6YU++&pK!Zk|i+Y zG37kNzfvVV^ihoKda1tuvxkh%ijg^T^?ZUh?$90Oyy3OU3gN3fBih_Y$GYKz;wK(+ zda*zOxhfO>v@BgY*u5AusR{S7ReE101M22IK@HnII3hv$opBfQbTDKt`5vxDUZ<)>*G5+eiXr?4^W|XyQ$uZ7Skm#A3#UBHmQe6)`}m6wB3UF;?dNJN{Osb? z)F0X9OI=-&3`V#is+j@0d7!f5ufgC3kh@zeVK&f}`W?DWm4(u(a{ zXBGU9)p>VW2JD`{MLax_gvTcxe(J>~lxHf#UH@>5^(kWzTNU(#?TKxA{`>(JK0fWy zKIC&Bo-FJU1!(AXuq}cOQ`oIj!!$&;#xzE@#$0ik``IXy(pVsYD`mN|8+1SNE=f;s z5SI>DXD=^Z72iMYRHY_!i!G?FPN%j|l;e6tX)?=yMC7=cn64Rbj}77xxX zys)zf;iKs=l~fz!ZfP>4e4JAK7EC zgPQokaRWrFf#TmPVWpa?pGIl&*@nj&*ecEO!d>VELVQdK;ZA^7u9ra zCQ*rk1s*R&+t}fW`E6gN@$G;SzfXyLk|(=p-X)p@DSZCC#I4KSqSFaADW$_0HVT*E z@u>#bsSVjY#9kvX+nuWY10PvojB>65;mCFAeGIPhg;8P9YvRuylO!RR3Fru476v+N zcI|*>ubN7oMat#|ZL5*V+_Hw${K3${l@FP)V!8MA!?K6$9j_SrV-QZgQ!Cl8c8Oio zmBLf2Z@8c-xoH%pEWd)JiD`}!3@LitU1h70dpLZyH7D=X?IQ4egVykP9eCUhON*-) zOW_X78-f$we>Z3N>*@Y}PE_oRSZQ8ZqGMivH~sq`T{~}G5eZ;V|9<&wBZjlj@z>v9 z2JJNad40*(kc~Hf+i>%*UwEUC3lR=jqWs5yyNn_TT2g!v2($m)*PZ>$lOMdyEhE1cLUY#d0q@o=E$LB+K`$z%3LA{?O{PoEH>xH~{F-aKjQ-!7zp0R#sAMLNvA^9fb z+5D0JzSM7jd1mN&KcK$7EXM!p+L`fA!t$ZRp>%V;88Ry#tv42cHqRJ*(?t%5Gev$6 z^^fk3R^)M|y}A4}j6Tr?S-Yk;let?wTXx0z;K%11!_mI9v_Ksyn!Rkf1GK`>1Hx6s zFK5(jGywyC*6WBUw7p$~&C;?VfbcXTfzydoTz}DjWiW>Ur0*h!cw8xyUH9sD-IZ5V zQZy)*ATA#ia+UbYEnqZVQ4FE^*_tH82Y>&UgOtYZC5A3qhB|r{mMX;@g*+Xz&&UR$yA{d#9=CFu^?{I> zg7ta`HZ>NRZc-A-J2KLt!{qc$JC=@*ZONaeK!Z^I0n=>d3a{gxZ49|`(<^vf#tCeM z_wUp6dno}P=6-{Dvg$gz;_4e6VuX~E zF)zbXbCul4B55@#q+j;t>}6QV)(2R#<8 zcI?;K+L523D6oxHppAS*$mm+iHdXZ5Za|qr@6H`8y+0a+O3rP zrE47e$x-EfSexN2_JGx4bktb5&4(6KW1ro8TS*sABr)Of*~u>O_^SQwKA--Hl6ld8 zmb5!nkz1RRYVE9kmB$YX8!p8b}0){9@Cj~3gN+@E8UpwI5l*vF_Ecb4qB zYnFEul?b;hj6{umM8LT9aCX?$myauQg?E4a zM#xbi24u?8XA(8_0YYaDk(8WU>?g~dpigub>xr}LZXb)>$d@r=wU8ILKidkNq!8^Z zEWJV0gL&U$Su|k2o=sZdZ&Td<(rf(wttisaPpUM#dS9L&y7tscZ1npUcb9J&r)LqB zu}V^0+lHa7GIvt z0sRamr3-WKq`_6Abm5gHOudObl;K$!LFn)1Y}JuFjpky+nz+TLhdW z;2@Wm-*$i1yELmCc)x76*>^D8y({P0u##xsh}@@Qvm+xkjNlNIE7v<;M@PjPIpRKw zGtB(iIJCOQdF(s0VPB4($t5;7_8}Vf>KIV+JgBzjaXLxwysTBXG@FS?L-q~jdq=7_ zHp^zsC;8m<8KbOFpsUhDX+nk;Kb-LSit?!8D0(KePNGS8`{#qf>>$G)4U#K^K9iF! z_FavLQ_A6w)NffVrpTWg-`A{LdRrzEB35o6jd0s|3L(Iy#-fwgj-hj~H_w=M44n8{ z={uM^EOU!0t#ufdj#V8f8MZIuS{n?Dogu#yJajj0)y%o-z8pXZ-P{RmwJpzR-KSlOr$x$f(~7ms5xF@yeWN$}pL{Mrs`7kAyY7|f*VH>#jnx~LSw zg0Aa;b|w6-A8JJ%U5`7QX&}LNt2L56h?qlhd9c7x%t`;d1~cEF+-&ji6f+s0-D{hJ z8I0H_li|~ciPe(^Rn6pA?D#fEK9SKFIhH)BF8^e^J|)$gBpJcb0GCj!Fg77HW8dl+ z?5}cW|JjAKi`A;zFjwO*Er6}`%U3cUQs!TStS#<3*^7uU^F%bveLJr86ZwwAiQmC216M9&LD( z%1H_mWG#jqKB8s9$ZoH;)#kkuWU(xcucxz6PHXamq<&8 zUEM8=bh){hZ`)UOIS+r^g&$1-bUv6m}|hR-7$>1PY`5UvwaGA<|KJN$DRk-WIP zL)8fsp*qHCqYZCotEL4l3QSluoIXl++mwC39d!xwdnl6$-1TYBljp5L9-RcxG{HA- z-aLx_h#BzG;50R^erU>|@P^B92G6`PVV0DOR*q6Gy_@q+w9D07Q6LJLAb)D-6nv`z zMvj2*)eAoih^KeEYAbab^Q3L()d*u6rO2-F)47RvHlSZ2zG}_7!y!DrZ*KkKe)dy| zHqqgv3UR-=bXcMP2W#MjIp+!T@ z1i5`M0oS7*bEB6+!4D}ZyM|%zFV~OehKWIEF1p!P{J2h~$b&&L35dn>=bveu2`doh))kfyL}{}|zUUq9Y7EERqN`c#qA7X(b{cU|-X{th5^ z@k;YZZm*ef#HAm!%Ld2DLVe%7&ygJ_RIuh!k&{Cl4LrZ4;6-}Izx&|>4mWuG4vP&! z*uVKzZ@e7o-@A;L9(~cssstXW;U+C*3R;z!W=3gNbFSS_0tzhAR((=YiN3sCWP0;{SaMblV+72KJ&>d0T;vXH>OITy5a!t`;{%DVg z%KvU(bTc!WasEJqq#}({9%;sqT81VkA`4q-hc$%Y8 z<)@q7+iWwH@DmzNA%#xTkLiH?jBRu zOSW{OJuM69^D-ixwkDJozo^@ZPm(!oLdnS=wofZB#5+a>`OBky#a@V{6b*p$Dt6ScS$#n?z*#(Lf%I>GZ8pY}*5uv%NR#RH z0eL$ukI9=`Gkv*`0A7f1BcA+xoE=@VX@jG-ZWp=6i(1u=#o}A#GxV>Vv z(;}Z6_*M-XGV+Vd)YkMl)V9i}Kk`K%ax>|rKT=_BlX&Vfzlv8vl<8hTbmg$ug|^#8 zSf8A7oLDm_c_lCCaNAp@izgL&Flb_Rm+JPzl0nWQ)K8@#3L4Ud&X773kFkxfbRN$aqMT;_!>YYxz^Bl|YRA1@4no ze&+K?knw=oZ;+sPW|PoZKRb}!oR3=`9t_jsf_JP>a!8U6sNzBShJwsa5{EV^pE3=~ z)lYexzKI+tm%IjvLSN|Xs@Z7JhxTe@gjH0OnHLIGa~os5ryN4pK>pM-1u?;q~E(sJOp&(D?KcFDKU!WjGG|{>xYKz^$O6St-DXgJajd&#wd0kbFxN;;tO(<#42Vq zH8bSf%xOg9$-Aahbb8$oLzK%Z%1$<;*9=Z%xUdcm7cx7U(~?;h>{4x4${X_2Ui3WY z@K|bQQtw6MJXxKAetG?xZ}!BYKBPC%xszh|%A0Q_<3M#SekDvnV)U#bq`rtWgG(0k zW5Ig$i!otRE^`z*;@Q;2QnPCB9A&;;Z;lFzd~?m(ptHJBaqBb+)9^GnC6|r_S|E0K zYS%n=s=FwL?o&c5#^w;1RG>&Xr!l(oq$)#KybB00K{S#gnFb~nr}16paQ&(2g@cJZ zWccb`pJQo~k4YSwxG>S`FCGy@778cdl@m2^=__|jDmer5#C1(MuDOktyNSQwrM8O) zXMeU(Q|8I3vYSSm?x~j>)LO?6J2Ks0)BrnsmxKcg2ZzoZqm?7F-4`)Gd2+3*(3lX$ zb1J4eZubt(^rbvsT@Rw|A(E_hDFfz7Z!M9(GFIZkvz?_oTyF1yAia(oB_b5g9vID# z{79>`l0a5l>IJ&##c6_(%`!8>Pvt|Xt8O}k;G&eC!%9kolgu|uTHSAY=*mTxlgGjp z{V;6iaZGTyCnKBFig`d}*YJMdoYuWotBc7^PJy^j3=a;^GvDiUh=>_Xdm5bEEDxsx zL*hah>9J>;y2vv++M6w#0}s|Wado@XztSWmB-EJ@tyM};UcdY-L)vdS9A0S#ZX}*; zI@C5|E&U-VZoSe7a(8Rd=h`gX#Nx3@tNYI8wPHgUefT{>*SJt(UGS@0BM})5!^u*GnB@wu3J&`Rl?ruT)N}X3*a>$j0GgW`T>jelj6>uJaF4IBoi&lLd%J4~G93Rn zxtDMM@NC>L@xVOs?m4|uzIGrCzqLxrD5x_#yCiQAQI??y3iYC$whcv1P4iZzSqRJK z?rWWmi!OL2V1H*C4CYR?d_1i00?~8Jgp_u&}Wo0hq|T z%X8sB0XI){N7v@X>YEMXM;sPUMP4qj%$}K@*|(k`eNrCzNU=C#XN^k+T5zNY!eg>`*Kxx9J364^(-_1|N8d5O zLv>gG>k%)YjN-JHHNSFnOs{1<(Pv8xA<=CMezsd5;VSz24I-D%>+A8nmk>(ZTQ}h^!4l4L#vo^Ka#&H;(St?Y+r{bC`oxAw(YzBn zo%P{pl^Tl$X$LGr55K#1g1^~d?K#MaQC2Jfy?K?N><*yGb(K57i13bIcpR?!2hWx( zt?BpX30f}={IpYZSm%w&??%*%0P}~Vj_r7mNBT)(1$*8q&;!t)I#&H7i?U(oNJ#nb z&G_q$9Im{csAB~9kf`WG>bUR^5&yZyj4r#9GR9>J)&6J8=F8S9z~tO|BS-KbrU~^R z2>{M4132KRON0{?3aNW4L+Vg4~y%h_|` zcX#*P0ETh5E=GpzKkdg$U>VtNh*_Q4nZMm_-&}z4c4K%ri2Cci{Q4FD|4{yR0RK7R z{_hMW3AuMy8tbn^e7jv2B`rQEv{T2rnc%0BHAbQ^yKm?Dtc}Gu>bJ%6ap)y zT<~a1GTsHX`erX^i0)`<7`G@yDb_S}MZTYHF@zX!>n0(-clswYnl0zsyR1}AnU=65 z2UdufBMSC1D-a2v9(eM8yIwf}vmJ!H_A^p2aIj|wNBhuG04TuH9&O>ar6>tVvJ!Aw ze$6ncYWKL}Q*zo$pZk;&s*1A>Tk$OeSF}$ldsX|V)h^$*6eOXcQM*MP;NVPsBALbM zZeuN4Sa=MK;a{kJE$vqiRq+ia&nz-eUS8CnUo4Y3U`O6$_3yD(c+pl*-1DS53f;zr z9V9JXot8#RRjuth4Qe;bkM~>T3!OHZGwJGv%q_Tv z;gcb=p!O6ht5q+)6*Qk5+RVOzIvW_Z#hl18EkTRfE_qQo^O+9Kec%u2>$pV$-6Y zPMAr@_Tf#!wbVBz%N*$0CJraK1QNeNvT)ny&7)aO;c1a%P`1t`$U97vJ|y>?X-=Dquh^QG}IV`Y!`Qa6ROnok$bfzh>gBc?CB? z#r6u}HA4MOK6L|i$LPqzdQ&5TfUQipUaNvcnf9fdJv9S2NA$RvnGHRU@-?V>UpY4m zM$0H=ZQUVW7yqYn_xie}wI0I->%8q+2l}?L<-{(l-l0>x?=MkFzA2*w+}q8ylvVxp zsa(ZBIGwmi$SDlxI4ZD(vv%d*pZ{ygAO35}JFNe+;xC}=UEUIGi>Ni-csr-!J0fSj zJc!n%u7zo-{?}q(JF4pEx$INo(M(NgEShJ-*U!^Q0!3hEtwO_C4V6QHGec6xtDsOy zzx_1Tmf z&pv{~xR0ak>Tw`AjU1&(Dqno|VeBO$Cc=AwOP}EI=LTkClEb=#rCA;2f2{1udStO$ ztY)K=sK|I)m!yKclCzmkz@aMuK8Z3ASGnF9E3j~SD(wNm5rs9(2b-B>=t=0KVfbSu zlDl~f(N-xf)gG9RbsKg=cwQhkpyZhixY;`i8=BgICNn<+>qO({I#t`)&9?om2QQkr zZioHYgS!{o4MD&)ydssLS+TWG>s!_!{B;+8$7DGF_)u(mwXZ*I9rYm7lU5$z$(cqI z9$$Yr`a>Z9fNLI25k?!%dZZ=6sYs@?My(2?jaETk$&yX4qA7L?kXWE2O$<5q?4p1< z0mjXnk%q~!RH4QS%W-xMq8WaQdRuQtt2ST#8b#c01WDUVtd&o|u0G=&Sjso%ngo=a z_iMd=^G1L6qp;p^QeqhEtB<87AV>WAH3h|7SiIykc=fY8#;X>PcM&!DpioUn*y*; zuU>G<(CMsQR3N4cQvp}jm1@mw-p{E4Wvm&TuVfDkr7}#i6=Y<5np;~tff5v*x8qG- zuaSXIUs&FtJT6*_6G(t~UH+l2L)iFPa`=f(;w^Gs7}mmtn~!?<8vRiUB+Ox!fMmI;y;Y5Ur|Xp4qE_;&NB9b3jhS`i}Y zpJ7n^H!bI=B#b1wA6I;AzO{`E@adR^q8=;j5FRi>>BxJX$a+h$ylSby&bJUqgy}Z3 zS)k4)Ds3@K=4}V@PQ9Y$f&Bb07d+M~uAlw+dp0uOVpMAn6<(*0!qygK4H6FC<@_0kz_aC@_Y zu8BzhN)3bQo1bQcnG^z84M{fJ zFG{zT0pkCI=;ur+daWXeAbp?qYy1$I zxtQ&VE1Wonnn*Is9e(BH=?)bL%=YQ^DjQ9ET}cmShv|sPEJvo^YV$pJ@2o=Rlb#*Y zT44YaA|qNzs=L-y_tJKTTuZJ5?cVCBzFAL0jf0=mYqj!_L?H(rxK`4B z&Rbl+$I*aQGHLB^Ds$SN6cX~vO&gTXWozaZ5U{(pJ4Qjcn>#Ljon|jt|>rTSnyvB0l3Upn#Wk-(r^LVgRCWq1qlt zqH&W0a(=k956F@nStT=kZ0_F*uG`eh>x#}NA$uPjMJ9Z!V~_B({bpc%`=xy@(vtbi z6DM*g5xPa(>k|AGQo&UHZpank0b?y3bIh~r|;DfM>)x(se<_1i%_~Tj^hn$ zAib8~aOt4RoR>Au@(x3ypB2wcS_! zu;F7GQBhGWOxDEtl_pp zqOx^nm2%0}#>Q=ni>cNV24c{}Q}gK8SI}cu4!Z2XzEuq-sC7LojteS@Rw&RA+}Xj+ zl1hFSN!DA?MwL~tqnGE8Lu-ze%1)gF1Q1SD{qEA4*EjR#+0eW$hz_4r>5FC2Twk5& zgf~`J7Kg=zz)MzE8@igu#G+_9z}hZ>eQ#z{{?f${44;=LZu|NMqhUj?=_Vke8_y!@ zkz>NVKeM#zn-uW$MAC4p%7ebTDJ!46Hzq`< zLJdG-<#@T2rTdKme}pqT?6GSA9TD5%vUvjNnM3{9`aS>689iF1>o8hnQ=_K* zcq%_?+3pIUiqa*!Zlz2fF;HlSy@=N{L_)^Z>FA=ylJc^?F(}X@z9BX)_E5ToD-(e> zf@MNxxlv|FN~0v-4q*9g?SyCh%8sjH8LHphSg*l1;nExh^Z{{joXnO{&}KMph8fU6 zBWz7D?wD+}4(}}$G*=r0Rx{0h$27kDgJEkFmpFDjvE^&ItKsk_%tD>I58T91*+eiw zGjf{!=Q-$bS>upvK2HN4-)rl-kWgcl*|5aHXj}Ghy)_OP^)K}{iL_wympC@9E5K)? zLks;dTSdPWV52xd=v|T)eQl%eFw;P;(lVm9_*K$)J~8JjAJL>on)S`*c;2_##6?7W zuoWTm%lYngCOs?bbv1K?*>~hpRnWGzP8F*L6s;dD)$Vd}u#tr%vQ|`t<~v)6!`0?kMCSfXbmVjN^u!kZ9~H)LKj4-@l5)X(Em-@f~BvU(>9nQ8%I_xi&H zgJmLwT;VgSxb0F2r~5s)2JA!Ynl9o30xyk*%in0oq`B>{jf*Fg5M2+3g(j-3bS^P4 zPK;Fgaxyz72ngaZ`dYdrBpc;_Q&%_m1WlvWN?_8KN@Ay%4j`hr()6V#`NM}7R(mjg z9YR(sdnnfQs)V=A&oGFMB4qTmP;ZX_9v_Q#*!sulSTuhfQbUdo-VTdq6QawQ`&nF)IzczRM^X6n;(n$C_`+D3_HSm=5zXf~9aE8Y>7q2F6;WV8&_ z+ggnSKf4!O&8PvcYTuKE0co4@!akCchl^e423b|~Zg|e{*1w8g`ZFK~5>JK;UH2vX zVN%NwlMhZF^Af4uF>Gv-s{jd!dN`!KU&xc5f7cvjK?$UTP-YQcq;+21vf?vBjv zL?-!^6{KmY^PO@oC$|7`0JY(p520*?&xgBgSfGGwWnS-?+TdEOGuj5d<_;hAW^zE> zV9qb7JEgy4LaSJ??nu*H9$0ijD3xHQKk~EwpuNDerlD=o!a$kZVPjt)0VMj(Z)m#3 zQR?I;gTr|Ub#}@Qxt9#qTfpL(&1M~k2L5?Qhm4~frs|5ZYkv47U`oyGW@N4$uy7d7 z3y5}$n2DpK1JNnMA6owi1DLUM8$_{{NHluvZ!n5!U$yF$ z=;v5~v~?t^r8B%ysYzQrbQx&t7`;Q-^4JY8r~snB4YM%(lEe|sEws)_?S|JpN+`Jg zoWuf6Kv1yj``sw5JXU_;f5%2T6dC=2Q%{n7+3O;%D9Or z_*HCq*6fMy(#&k0P>@bc{SD#arD6~j-r1X|+F`U_5MAmb>0PI?#z1irN5Y0=m{@5+ zdaOa`(+c0pJ!uIQvC_=9t;x=V2ZmRWG3J+-fwIC(1ZT=Te(Rchc_OO;9;c)~I2X8| z|E`}v`8N?z4FJ%~^Bnen{}pc|YRJ8tF){=BKhNZd(V=uM0UxI4M9+0Y+r`X7L&i1q_Vb~Q%!QCO@Q z(mqaeo_~H%K@z}2>f&0`-bP}M35XZ)eN5SR881FR26Y1vPGh*b(=T5g;tg1sGKoJV zIPJzVlt75!>+63mHuK_S2pK|Tay^d5O4~D^y?6c_&%ehX)5?HyR#tkB2?s=w1(K&+Y7%;%Oe!kB@4B6(bG619R=y+E|LBn zQxIF88y8FR4@V&dk6QRVAfY6`ZrD|{O7Ln9h1^j_I)nXzzdpb(kBq!L24zwo8wC8E z|6x!5cAt8Bj)Q5M%$yPWuf_QT7c3A-q7V>kcL#pwZT?J`4XY7)2uFA=s6YH$Q7P^dsqD>6ZIugktm z1Er!DfX5-0{KTq zK|e9zu?y~ZnO1a07DTOU9*KdNa_!~bAMe_}jq<=r>%xVfW0{PzXsgV!G*D$22T$bj zkU|_9mPjOVKL?RS(tJV|Y=Y}W(ZpC@BxpaTndE9v327TDbjjaYu?Z$8Pl_U{+pu-c z&!yZTvTT@`T9-mYJKPxFe}B_tE}OA;W!m^~T`PKOE{8@j<1oLXBiX||gxDQEHK|jD zaqgj!q%8EIg=LsS2>gpF<_Zq*dAmYh?t>Ch?;z)6JnHOv4cB>z*RDGqE>M$^KXx5S zA^K(9NB0<8@%a8sm5x#%LkaL&jbtcxCd1M9jhp9UOAPgT;A_~%_cTjqd^j@U*nt*ia z5Sj!6g!*ng=iaO5-uD~dkMHL#7+&iyN8!7fVR=5e6nRg9Q$5+QeUQ@n6gR zrMB2`Dfd2_NFA7Ln5wr6wVkarx`s$NMx|EkrTKd3Gn%0|nHJe%{x^|pA(OC{9iuP$ z5AXhU6@O^|#2lxO7Cz$2X#dqMqWpdbxr!P685>Jmu4kGFok-boaRhkS!gS_+9NyE= z(8#tlALn&EuI|VwQ zqwCx|vQ*@33!bA_DzLMc$qZMPa!gmx;;U?ZqLh8Bt>(m$cUEH@nlIGt8yC%|s_lvf zJgt%sw?~I#E6xWeSHkh3$rtv9UJ3vNw%SFK(l@}bF;K=^0!Ws#PRqY5TnA`>aqbH~ z8o-nwGNY4XD+d9eOw2}HPcb4Z*H^cvV1AWu6K;kc%Uky9z@YXQ&ePQd3eC431;I5X z3a%(3Yj|dOfkIrLcJGGTG7V?>mz7}(rmS=?$)SSBEdoXXyFRLbxH_{&ZaCj0lJK|K z^gZ-u+Af0adibR9Xr~6;i^azs6l1o!PzKotWdT;B#}+ZqydM<<^SE zg@@1V*i}tay3b#+{H8ci{836q@m@#%DZD45Zso92&~U61+KI719c^6!uCm7B_dU<| z^h1~H2uh<6+#hcm8y8Z?gWm?c0QgStafsA(*os2sS3^O;{T9HGX)qK2X_^6O)%to| zvnmh^0k-KYDi*L3;Fzv*cG#SOe2=pzI)i^00I1lpm+tNr^RF75~pQmB!cREn&ItrM~vl$K(1bi1sVX${to73f=Y zI*c+i!)mN0X!-f4U^!YP^||e6R1G+P#Y9PCzU-CEDm4nrV2<}f#-ZMO?rABgB4QT4 z${WKnG0aM=F_7w@+7k6HNTz3`NLpu$^L&)v?0DDD3}_#Yp-aifS{9ln$`834y~cax z(`qLcFeYZ!!M?UeD6m7(ZI{n47AI(spCIJX4|n!BSGV3(&9?w}!dQc!Ob{TTIhX+1 z-J<>c`SVioj=vx%f3RnF-b?O&0)tfJ8Z4E0^lv5EUV*bxoUE`)EgAZqB+@ou> z*y5DQ8c4d|y&GR6_4~c6*v#;HlFZOdtVKN6x}E}7GdyQ5s)>5$fq`jUCl=rpl$#=1MwpuP((SMf<++wHW-^QP#bW5$7F4wF`xQjz z&@xFaQWTgr5gPaAKRIeh)+Zw2wao5gqBdLVldH0^&R@?y8a%vH?@-J)R|B8v*yn-~ zrrD(UTxcKLH{}3~$Hr%?{6KT_uWcOQA$%O z%_1U~pEmKT9h?(j1)v2*iYs8Er6(;-zqb+1HGl#}TX>;Imt$$mYARS)Uy}mymw&TD0KahOJS9%dW8zDmm5N#hMoEjV`FdXfn^+tu=b?XB3g~++4Uc zCZnFK*%oc$yv+nJ5b-&!v{Wz*oS)np{5ByoS!NS?gdC`#208KTmhfxz(MWOGK(3gt zp8S}<%N#y4vO;#bE%44AZH?6WdQg$xoB=Vpkt|)f-{|z? zP$mPKajHXDgTP{5OsJRqxAm?@%7O)b{EF)QoIiw#K`Pk#HTJhaljn2=&rhei2RBWm zLX#$oV0_Mb1y+z(nOTEp_pK}Q%>;8N{)B5B?0%~Qu^B0*Wor9Dj*+W)Rpk8(YvuBB zOhpA2ejMYst}cenJ=Maqx96*%{Dk(0m@K z>mUu*S?$Z$YVy<6-Pp99EWJ`X+MBUC*XMQEs@(DFD7+-7jEGV0u&%imlO6{&mJ17f zYc@>M>>Z%q?X&vT26qm^&4it_RM3^cJbm7=BhGL6*pFPXq;j*tWw7qE?#4^hq;x{j zvbEo&ntu|?6&Px@iB9w@WIsZbyw&S68`!uv;al9~+0FNi$%NR;PHPpGqY z+1gMWmdmmsArVc>R$}uWRlJqjEYyGYj;nx!J!`_qb;2#}`DDs2>6Tid*Z5XJ^OYKc zFLtK{^op$>nP5NeA8@9mez?D=l9J`6ME*M$q*eQliyxP4Nln3-_-TKR?wrBS-ZV2` zMRTdL4kE{(x`_QM!rLi7`3{3(AU33Jpb=K0pyGK@GRcXjud37_XnuG=w}@Ws-b-Td zM}*4bv0IhG6+Ua$jF>RZe-y#n8W|{Q{GK4)1GCz zMRP_K6&{-@C%Z!MWxL5 z@6NT;$@i_Ms`I0gyvF5P?3;Eqvjm1yOxJm=FJcD0h$rTc6Avt4daNy%M=`cDr;&~I zC#-R|ODKWvDJNMSaH^BlvSach5;A;s?Mf#X-dM|!j9m< zqc$W)4iVV(El|1ip`Y_ihu1Jgx2Y+%A zC)uBa%0>{CYgLRY8+et70u-Bn<@2IBYu~+Lr5GG*CHa1g7d}x&Do)qKo0Am5p{%cR zmkJCUn$H_-f`ra{C;>ktpgCa^h@b%+>egX=s#hr?GVUy>e8ss@gZdS_sl#>Yg2L!R zH8nNl39dx3>z8d(QO^MyS!m0ybk*8Mmia{=&;z9MQi|&iii}rE;VtCh$r|K3JdNAm zzspine^_t6107q*)z{2BW*SX{>G5nOp1rV(u4BVr>5@vhn$Moy>V3DKTbR8|SD?k? zI;X9mhDq3Bnkwjycv_|D-G(WZPh*CtRL=0bEUViF$^vtx(b24~q=Eoexlt;$(acE# zxIWOlhjFPd>(stB<|;p4@N<_nx4Y8l%vDbN6S)LSFzt^Q%?Xqj*^=D+EU~|q(wF;# z*-tl@zWKDU5c%U)WG_$vz&6I*v%w3Hftr}%{HVfs8{55CKU>lN22p;{`it_GB7=vtK22shmj4k*OC404}>JjJmrT~{g{6@8m2+q{J;(s8JAhVyE+S>VflXVznt zWTA=Y8>96C9BGLjeNKIi*6Wi-UOz#L=fKpGy{_ISyJ>fClryIMOAA1bzRgvTfHet=;6^q{rJ(FD&p9%@mm zh@Na1RYM}9>&=_Md7DMi)-{GCYM+$q$8g(`946j39yRLpTDLNy|O0G%g(0hf67(z<5mF0 zV`00o3yEC%PvG^3T4`Pvpg0l9H}IKegqxcrtID6!{aOfoW*DCe&je z$7vVPy7Dph39q=cm-NWSW6>O82{K_EKH{lpw*QbkU!z&@jdb(XCZkKh!Ck@PP`jqT zTLrBtKZ(Ea$bS=mdjf6_O<<2MaJt6h!3_ZA5$F81Vq2dWqwcY}?EGf%q>9SD6x@_j z1$#Ov*EEz%M>y2MWMT(qH@X#!c$^4jNsRd{N3TLY{CeQ%4YR8|UgW@ZX-r8ejg+eo zOYjVD$&A8Y`YolDLp+rvp9tHTHj!U5dCKp0H-rnv05q(}*dj0O$Q?x(y5m##P5q_0 zkD;)jiZZI~5Xo~*!UST*?fO47HvjZ`XGH;g#P`6~-WYlqtey~FDMUY4o1#DaIFZ|| zvDomqAz7MHhl+}7Wu-TU6SB+AUnuRn*5P)vt&2O3n?wV0Ueev{&sS|O$J5eKAmjJa zNhcSiy0Du6z0u_R^tJThhpe+cdgW2=qtG{Ma(}@H+ZfN{O3CmZ{$8hIn5wkK6@(|s z^Fg})BClK@@z29vk@|_B89e!a+bIG#@^rD>w|`5N{|BZ`3Jm!C&uN8!ycgpMfK=bW zl_!4tuOs?-SpVan2mpWz%aHanz!~7rU z_f2U4n`wO)9r%C!&cA=~h8p-|{<}H(@00!S-@ic(FbQA2kh|v1{6C(_)+nIW^Zz$4 zf4eLHb2@>><^Si)(f{Kct9!q$omPpPWjRaPD$pCz-kZ^iDI8ygR?Cc4i807|p{qsuY_Re&Z;{1HI7doAmWUXDa@TJ?S7 zHmkInGqQJpB!sE)9(j)&43PX4U;bXNt=$0Wd~D&cEYh@9oBpq93O%A=-geQIhdfZ|wUUmI+i$wbyi2E%#|CRRqkI zDwv1kvz61UG??vHB7j6EP*}Y7@|@pN2ZZ0wPsXhQ)@wC)OI1~4PIMLk=>ZC3RK0|; zDics})-IHukGEM@l}M|d^?L~|*1u|~Ymtq5;#d`Nu-?qX zAr;zN2wM1{Ji*&kSJ!xUd4F_OC4*B%nLagIR1;Y<#mbuV*6Gw2K1qc5*0R^6p6PKf zMn~8ck-Z^WJoSf{ zPu=yQR#Tbbt_$TFExT1XpK*$*rA1g$YR(xKpqF}d5gGQ!)Q1Yr%c*LqiF~zGhemXvd^ub7e3rYH>)xI^89-?aC5$nf zeK%UZn$faC0T&V0<1KX|ilZrkPLr}r1kPC?u4Ud3a68L%jZDen!UG9T*DgspMIpQB zd_gzc7>lk5MGb7>WRSaMR|G`;^29md-Me@B9a|-9$qfQ7TEoqjuBm*GV1vmGX;=FY z`Fy(GyA;h?v%J@>g0a}7%W$&}8&gwb^icy;^X(~a(IS?0hfM=7nWj1_TmGn-6)?$& zdmI@aA8rjRrR@?(zYG5`hbgzN>BOfMZiE$YQwWcx@I6<)hU=Hh zEYp*u-BaX+ws^O!f>0O+!c?fkP0gZ>4HKvBZ)!GkR~POL8wC*55x_|sq(}>MsqT~{ zjMHUr&Xmqik8YP)TD?3Alpjr8x^B2>OyoxmhbVso#B_A+?VXAaH+=F8Ksm`kVwf%$ zhEOZBv$tV)DAi%YBIb_1X1u?ZKTLn_`(l^;b;j+xcl!H|;}rmhHnZ(SIVXUzl`Maf z-T2HgSLke(0qwKHK2+HKlZN@PZS{vEAewS?R(!}Owc&7jf(;EoL_Kl+quuyF0qyE> zkF~mH%|IH2H28s5FK^nTv$Y0xE`zZ~U7HGmO<)$BE44!SlZ?*Mrf_T64;a}1z&w?6 zLeg&0?rTrbv*JCr7moL(_aWLG7WCSaC+m((vyOgS9|jZwRT0s!y@aNtbfU+FxT)Pv+Zbl5pu4-<5)y2jc6*1P4DB#9qVNNnb1v^Jo_z^Yft8?4qO zV&&viE_AIsMLryno*$hF0{Mt=jMLi;&*)Wi1<$x^A`c2|x(bkNve4!<8el4`vDqVM z*5j2%%#)qTWAbaUG?zV}AkU-2-h*tPh6dR%Vyw$w2FqLh?#ylq0`hg#PybS;5jfp^ zz4ts=?Rd5p^Ium0*bAsX$*4`_8!5~i)d>a20u6lAjdto4fW;d>+icoKDIw%pdo>a~#hZelxU55wn(PB?2v%RWA^&;{PRTl3wU#1lfaz4{g`6C+pb+H$M}-S{+Dcq zer>`g#%Ts{v$wgAhR@eAzWAPlVR&qgf>xhe+?R`ud}7~vfGfT9?DJ@@>}3A>&j8(V-; zke{a5Utvr&?YH<@l|hmN{z`Fw=8-4OZ_)|)ic$oRzd^dbi({u0m?f~OGz3~C0rMBL z6)|Rc$(OEx0Ladqb;O;50-f~%14{9)Uy&mtkP>Q+W%zWOSCf)3PvkrTtlI#%7S63T z?SLctzBF~sH#57!G>#woSiYl4SNfO%Kdc&7@|dO}OyOw)*#ph&&AC^fI*O`5sMvZy z?ygKyraRG9)h;7X@Bmx%BvDQgu?#4C&ZL&=l!a8*^0@B6Ijmu8*EC#hFgRbjKQOd( zhLPt*rAOJ)L?)AVY32mF@n7O+gR&rDJ=b^gAqYl$2fGRRcs6}1cULRw6kfY8^SP7w zP3z4f$iu|$LA`=or&!Sw*6%ExoTByg-BP%lcj*|(Na(=@k4uv5x+BPW>}m|O3r&kR zs+4|JQ4(uy`Fqg{(_SxH)cV`Y+@_3&$B`T+FdxC)|L|6RJiehlaUUfKLTN&;hUYz7 zWk4===WC54FPeEU>n>w@P%U7WVPvsx5HZHeBBR-X0g!5 zOkEc3e=2Euk0@fKbCIP9xWP-!uD(@EbuJc%J>0n!XL2ay#vD4lZapdfZB;?9E`>If z-pC3~EU89R`W#j?{>am56~v*^%}l%5pckk!O2B3ySp~trvGeu|Al$AgG#o!|cy0Lb z<<)uykb5*%{~?@f5=FqOP8x*W%?Pq9VfF>kU3<$7e&GeqJ_KlXQJ$FZRN`c%=QG}z z@Rygy7$V6c*hcA&W!96WXZQCW5Px2`nl3siAB6IqA92;Y`)Dv~r7SO8U3SWhuQ9G7 zj2{rqFEkPzM2d^e(%f}J1U4YfI4-h+1G7mHXS~)cw@W&jOP9t22hsw4{PspsASRGZ zhEV{_eL$hRBQ6N9tsP3_ll1i34D(+P{|<~FjaC-}eFt=5&)9?<=#E(B4N;n5xuqB1 zZGUwEQ)j-h+RoEVGRY<`tT~{-SE9+4;dVD?r9T@l6?uw0aFvE*_Z6xai;?hI=i+DK z441!-%B&ah=<1ek;IdH3n$$NeyX@W@sqF2?xM!4Itp19_XN_hq*I+7S18?L`o-)~F zh%AKj!DpsdB7P9(*+LcxB_wT&`3@7e!-PDP_VmckyJ=0KWE&52IG0J;VnM`#xu(Uw zkcHHFZE!?l!aFxUzRY2Pjr92mc0AjeBy$xF`KDdYA4XMP@3rVMo;xZs`txO$_{q3v z+Tg72l@Bi$JG7j7-Ra;dW%xiVB|YCd@SRb>!K(o}yCcu(lia720cPmjR$%hotSyVt zOdWqhORdh>TfT`%`Wmx;35hAQ{OVNl2mWr6CX1RJ37?&qpBW?MbU+R>r{<0%TzRQb z{`BaK2V18x3ko%AdRxFnw;{AZp3%y(YF)208m%|7IZn@dTRDjvlizvm>x5kkbHY&7 zGy4HkGoD#RA3nqV`9y5j?a8!;nq^P?8MnTUsS_|%nc%`x{5NwOm9#&GQ-mD)lq`KM zCliDmM1uEnLscRg9Zl_IhnG`SI=?UCR`LG$lK7idV6o=aNUlos9NJG`gHSo z!WvmLReNJbmZuHUd9^3JU%MUW!)r&!_bP4_fYVQBCL1GXb`~qz#3xybF{vcnj^k5W zDCv~*l6CFsx#D1_q2tQna~lHl!2`r&j z8ZRD^aw_deNHA4@M4SgB%N zPPY(mqF&6i%q(TrnBDX7sCq?4O6rACWXv`r)ADeW!gmr&pv3SJJM@Z<&E#wWT+GQW zlDBOFT4gHo^hzFU4@7tDlxIuGT(%3NVqKr&lT5+~K|X%FRyp}Fz(YcVSsJt8q$-oa zjOyQaB=E$)Qnq!Pe_%qubN-!(4YJW{>!DoADsWQ{`m7fIdQ$?=$)j4+1l_+thLu_L zIQJgYe$khI{Nre#H0XYVQ|}X`y=}4EtDW?(N}XHbJbmm2_4x<7!WG131OIee6qCpK z?#V@KrB4wWqPAtABmdhfqxqheDaW9+UMmnDcUJd+f{*$VZ_wP6T-{jTsQBhNkwX2UG64ylzwKUC2 z?0!DLeSL}l1F~@r>B0Xf=0*E@X@#Td5b);u4j9+;6fAwOud92yjg*o-SMyxeQi zpVW*=j}>s`9a^08q5VD+mdGu?l7mm`mYb@T4ri*PNcrnlp0+kAlSd3uzK85QXgEA^ zUJ(%$k+6vRNDA?xM$-AUyT)U!3(2+0VW@H-Xh=G(=tOUzM*~7eSJsoqaOxr%m4wjU z%j9&TGQT3 zGRg)BxOTs}lqmZV^N8DOmXr@vtf$YBuwvVKAl|pF?-ahYwH`Wh@VS>Z!bzRu#;z;V ze5z?W@v%UXf3CXY?7C(>hssO6*)94%;;7_+>zvXm(efgaHRR?$7Qpq(@4b&}{TA*V z?=93kd3lt&*^@4O*L+y-Lm{Z}72qvH#j<_#1$*i^8t~P9O5+h_&tthfn*zyiS!BzWX%^&?cUBDUnb@&(WAv+}wD_yPd>#FV==yXVG7OpirH4 z&K6-#=KpEKqYL?f^?!W_aDsn%q^A9WpEcAR zkShASV`(e(V+MmS7|1hxkL>1c$>0CVpNCj|z14;ejI+D@ooal4@au=lQo7>)7)+oL zPpf}=v}>rC!LO!rlqOEC+HBBy8yDPYD#B}r#)213Na|ljFz9@G1W*D4XcY*T#&Sk( z?_jR5e~7qYs8(kJy7r=E)@JRV_i1ox04{fy%CP?rB(e=X;{1fqmP3KG0ed>j5VodVJDzyvh56On;hEgM+7mmHO zlnD}cyj7nH85-wD<0b}e_wO0>X-8l z#H@kjvOg$>&mI6)=wiW{v=8o|#g!jGcV@!0^VJR3hT>!x*V(8bN2E^#y{H<6n=~)9 zV5O*=f!IW^4ta(Z)3`}MMVj3ccKWKSy3ZnrMU1o|fIYI=oTP`Br^#_T8AEPYI;Ug2 zr#7{Nf;8`*txlac8C|0&K7OC}TPQSRl%CN!mD@2_E5v^KkQre729HwHS@Ki)oqR0o z`S{haw==@KscH64eM& zr|mC9pMQv^l}`yd`G+l~*iE;4KN^9D)iJ?n9FY7CF)*@PawOyF+5UR_f850&6?S_P zn!q9k$j-7(T#mDg>KF{8H*gIapM(SiVKXofi`NzvD?pC@<}Al+)wrk~`ZSBwZ>7uY z0=~7P*D0CrLgNhz<;2e=N*CwI_Q652_CV~m-tq3yA{oEN*>n8b+S(;Rz=7`Z7y9Lq zsDN(;#m@oOl*Yh4Biq9$#k_}xWXV}Nq4RaPq5HTpi{C%{Im(u(6q|M4v5V`2!;;w< zLi^Lbl&31t477W|6rYf%H$3&sWO@KQ4rG2?GT3YT+v6}IK9~u)$zZY^BTsiYg@4P{ z6sEhod(KO{PcxNPtI)3ISFckAA@{7|-bQdGDMiHx6Q%kkwL5&Vy##Lu*-Y8UX6}Af z8ARu0v~&k9_;MD7_jh$jE=NRL$pc(f$%IO(P>CbkQLtE0X>HK}@T$asVH{Xj}7I5z?v*D(8m z!ve;>`L3t9NUF$v|QP@#s<5=p8 zS;(WUF0ZoS-Td5%`=)M%7uBn-0iEX#b#CnquSX4bq>n$S3-u>=?)(_cXUMTFiTLp0 zxt54X-Rsppvscsf%B1gWcjGEi!-J=^As|Ta_&CYa3IeJCr%ni9yZT^*?@3(Ti^&z7 zDcEtR7gm-29ANvX+Bf`gV;b!@?c;c~Et1IV$*XtP@=m>vShEYNRYfqAF8H2+y+*ps zg!ac2Qc5qFwOlf3X9;0~)35S-IJrSj-+GXX;$j|7QN@9fmTs}f^Qb< z3#qT@-H*-ngQdbV`+8@&T^~)*jPmWw+HF4!cIxwWvv|)18^ajzClA)a6Is0qgIG4> zpQ)FMojcQB!%9*D!)z88Un@y&GVFWy73$04F9Of}z z7P6vQes^@nc};#($f0Wo|N7)zA{#e_77S*%a!g>nBon{w=VF<+J3`96>q1Vf;UTx} zm}8;Etavqh-vUvhJ}IB(HNPjpF~4%HF&B-I0Gm}d8-a})AD^}cuIXV<_QBe&QMi0u z8C`qhyw?0EzjDgu4fzDdpigf~J+dI|=jDqsB`~}?hn5)ZjAPql)4T^q! zy=G>D1(xe%fIu$4*e2zC*dPB$TbnN@b|_>lr_W>u0e0acBqW6S zl|4S%33ZNSmRFM_1PST7yYr>3ll-1000|8rkPmy#EAB! z<$2_&IAsI64OWxo{aQL*KUh>hJ?m=WyVD(aHVMpVd?S^|b6i5e;e%K{XD&P}9PIhu z(r_HOf>e}uoT{e-zNO_UcQdXI+?kFxp6}9|kKzy6ng_@L#zW^x-*RUsf!&<+dS66s z@wB409y6%dK8Y{U$u>l+4W(gULfvE^-32{7NW0bW3ZSY0HsTVUOcyFDd6uI%0=szz zVLpC|Mhy*4o`ROGsy(x%%$TE@QlUf|>_&~ZGvhMHjr7oP&Yu2QqrGWBkV=!+(E3h! z$8F))FI&{i)9c*#N|U>N&YS$$o6oK5)Lhkcw}I-t+Hvr~?`KNvl=A02X=Bnbd|>|K z1+YJvBDv+sMc!U=ORp#X8WL$fKDbSzWmzyGBIm`A1LiI(2rilgAg8=nhJ81>VU7c;g@!WU&;OQEqAyaJ{~*2jkNNp24i6nsNoK@?&LKWA3g=Ttl>JF;?!SVOjJh7AW|CBJ;x^0_a1 z3^z?61bHNK3BXJZRXZahkHU$$r6RWZ<3eWzFdDq&n4jmA45-CcFNR0}5I@J^-01Y` zt{DEhhUo|VovVv_BS%MZu?9-Aj2zwZ_zw{g6Q$Pf+vm7Rc-Uoslo(afg%B`C#L!3R z>PHNg!)>&z37dVq9=SaG&(?8m%Dq4a3`{LJ+6li6}^RhEj#Y zjT~G%Pbpd?#_8CpfeNHtKD_fVDaIwm$&5KuLyRz0z}0}6YNz+vyTUN^`OctYQTYR& z-h}hys-lrbUFN3vjhUK#R5E*2xpR#AJm6gLLbtxTBsMbh53KG~PscJ%8xakK{%U5VQJ5Miy9d2V>*nTp!c21DP} zcH!G-$VEl*J>&8S8A_SoWp;0fR?NtC8qZ@e^INFA0hnKpKsPp?D5mocX_g3x3?w5> z7SFz(jXyKjOuY$uSc=03GIZ=p^D0h8u!g?P3vwyn&+?jP6BzexQ| zTs`M2Q9H6nPJ|hxo`Zh(WuJ+WFL?J!O;p`pWud%bLI3@)>Tswz%th31H|t>G<}WpJ zA`-$7pKgNQl*I{E5Cw~-g5pEh7#TSS&iIpS=tQ+6W^289So|@{-l2MTvkWhjwdte+ z4=ZTF#VQpHv=volgWVMggb|(xD-s*;osW`rVZr#e18GloE+;eFhNOI3Za$~*n$OLg zP@d%0j{6s2!Wb^Bybo9fM&D$lV1mYq43!lDuVO@?qy=Zx_lX@M^fcB^iE zOc%qoS%0SA4rG@*M8WGy`tGs_R`-=gE1RRrCsk5`tG})-njh~lo)_Z9Ep`@iHmhs) z#ZXC1)AQ)YI4E#1P9$qX4dtFms+-DM=p(Z-sh&c+<=QwH<<*F`tUcOzXnBL2*N5pQ z-l2Z`M4;aK(Se~O7&Jv3wyv~M0pwKGyB{lcl{+n`XSMV6K}GUpQM49B?+iu5u^I?P z_R%EEs|rc`rs7?Rj7?qK0_k_g@up>VjgfvlSh=(9rU5q3ls5N#iRUCjuO_p($79La z-yc1jvUj`x8*=j-ibI3*x<7TwxwuJ0w@K(R*-l~#JH$yx=4Lepck$7m6tf9z~=g#hJ}VF){fCuG#k6`X5>DFg8V6`3TMD^*ZUR zV*ucT039WVn**TYa=7ZrMYcfM`^ANwon3PNi(e#d`!l?#`xvjG8ei$MNAPMV8E>C5 zVz>MF_K|#i+wKh2c*Gk-B(9+MGj?T-5&9mXEcs3Wv%sRo%~0`rjAZD)&D@+hm$9l3Gqa8S zAriA2A<>83O&KDiaoL9+<%zN6TqTh*xxH8Ta+{DqNa8s}HrMo&;dXkL`<2>SuL$r_ zLvpQ;C#79-wo)M}dge;U-UVj#>E}YVlPc+s)lbi`W!_?-XSOW^Z&d&@o|KsErJ{>Oa+hA^MA zWhZBJ~@{wnPs`l_%Gw71{d3r9aPGD`ne2N4Xx zXZ2^q1<2FKiYPq{1dJ>yD(bo0IfIc$Gx=!?Dq`0HYX9=;sYj_B8Bhn`=F7?)q20jJ zEIALm{z+f9E`wh7bGMyK%&EpE4gkQ<`4~PcL`tK?1EncsoBqs2Mrf~~p7^auS2u3WsHpkZ`0`{ax zC4FCsRPDHkz9RXjJ0P>(vG=tujNIa9H_@1xuf()9OLg$`R5CjHK>cOz3v6?x@@A_( zIqPDmPoD{gg3>kSwWXQ%n6+bQB#5<3b+B%b3A{Rp4?pp}AkJ#Za%NG_fAaQbLzZF^ z^10XHvv{b^+jg1DaVKk0{kxaYKPgqx8^2X;v9q9UKg0R|boC;`Lh$i+#ShHv9est{ zXji^v|0F?($CNKLa^6G&_*!+Z78%#u30{d+s#;+1t;((Kd8C_Wi&xOGj+A?ngn?{} z_u(&W8^g&_{$5WG{yfF@!rco`&n7Po>sTq+o3TvuC~IiAYx3%qOFO9uKdP<`wlzD& z^{v+G*5%$Lg=-Z0TQ090y7&|FHgrEdohFUa zB2OPLA_6F0WCJE;GEw}_1ifuWhPjGKjw>7DbIS8=P_R^KY8|Soq`vKa=*B6T;1w(* z2!<_myt|+++K1=gBmg;>e0w7OOZ5Fnhw&?>c5n4YXY5hYBOE`w1@p!iaZ8-;r9y!D z&f_Ui{+~Ob&hh>qg`D0!Pc*bchYMoNv#jF~-|Fo!?-#a+%V1GJx2l40R+Bn4Ji${< z7Tve@m@8OLli6&0d)Cshm=mDw?~d~_U!dgyWSPrVZj6*(DYYOjGllmeWT{i$7`bI^rt6P} zESEZu(9L4J@+dz0?7jIArs4@rq|ySO&i)w!DfKbdFglbR3;+z$%w-q!ImdHaMIIwA>cZ?x{6g{6#x~ zBP+-YsQgFSg@g=rle#uv@4)3!er!#K`>OF}eNX=?0iG#50Qx*z?WG2sr*j}@0;$3# zf#V1Eyp!e_CV{^kLF?wd-{3nySL?swyAdPMX^VZO)p=`Ch}tJkcp>afo;txp@@ee5 z4=)22_#9e;aLp}kG(=m@`y_$|1Fjq+kv{UbJC`~6U4~ZB#hNU&_S`^}?Nihjv!8&SNhf-$T&foNU3_@YPbZXBpAS*bE_$Vf za&R!%YP;)@x#3gb=-m1E>~RAy#+xeOtnSXNSNYs%Zw!}66C+kB#ZO|8 zc$n@RMU1y^^e7Bt}UtHh-xPh$5kwwxxqKK^gzfxqv5(5|pn8B6~Ex*hc@^;Ibyl0)bj(`j`)< zCtH#PFFe%>9go9NRqtusJT*fw7=ftBJwP0Bz3aP8f-DA~;~7E9S96f>EfhM3OOm z)d5v~NmgZ|GJ97>vcvP!{(kR!W^Fc-agonLy&txWzuPwD!vDOm{`cClNznTz1R^=15a9a4*Bpy)k%QA;FFupHh(C9=~ zesbb5xa#b(BpDgoU`}Gh4TzI+;146?Q%n#AH?X}J&KN(I22?j--WgwmW4dEaYDk&1 z3j^KAmQPhOgk(hDcp+hqhnn)Xg#0Vd{(%7KJOTpK?pKh%iLd#pCoV$VBAqX7zcGm= zWE3#CH93dGII5E&yOo~UpQ|h{vv`fbeSL7{!nxqt=i*zm-don?8pp|>E>|Y`P zVDKq6<2RR^JiQ>ylg7HsaY&>UPR~W_AaXqQR=2PB&Drj5i<3&)*Vq|H4}~sIX4`+h z6xt=9|IirRb2p?ouPQ#9Ny;oea;$se)C)!WCWViBf2%74mpgb5L@OR}_l^nt3Enp) zeUH8y8bZY#g?F)(HroPDvURu5m59KB1r8<|5d)5GwN~;4TAB&q=e8gKe%<_gS#XL&j8N{e~Gk4F`#<78e%H>^e^l*tbSka@GQ?Hb)*i zPShN-7j5KPVZHdin!wy#@!EgpYKIR%wi&}nkDBAJhFi{_xnIg8iqJ~>$)AIQ$(ZG4 z*OS!`cV=|XH@(Mm_NRtK#e%Rm`Mw0P;ig|g?^t$PhOU+D@A+Rm;cRSCH>qDd$N^_b zRcON`QUMc;z3@v_W~scvlh(FA*rQU;qPRPV=p0N+tpbys6m}@oZ7}tz#Gj2*4Ha6i z@BZl`6LjRiHkbGh>{sh${}tKDHocDJe^g_-hlOrmVE<=XB` zD~^DY&wZYBPpxi@!9>Qa>X&i8)d>6fM*kT^P+l$~DDm%>`xt3TwXLS8{Zy&5_C$X5 zMDCpqNDKvL#u0Yler|_?6y>2e1Aq37o3k?O3zEA}k4Mn=o?qQR5%xY5#$%ShN52m* z#J~3r(3H_-?yBDVX6^x!80U#q4zD^?qM-izN_2q zNeRI`0HYXJtyJ;OY)EsCK@V?x>73w~#Yu|&`csJa;fMX{!u@phW^<26#1gl?^3QL{ zaNb)A746y9f8T0*n$K$~Q+ zvDTBM&>avLB-4q|9;>?RefU1637Bogo~u8{j$_sW?c1mrPlZV7C9(A9s~6NqNNnxw zgLQDCz5&gglb-yhN6B%Xc^r35eWJByAXx8WtHr za|7Rd>vs~PDDndwv00Z>s=%ARM+ktURJzWqU-zw0FQ-1;S?AlGxnZreCiGT>H!$e* z@48Z!U2-Z1uh1b}yX`%m;!mg9_(pS5?(OA2i-d0aT_%CRCiUY-1%mXY^t9AvTb zqcj#t&~#OLcZQ!GXg3g3d?+BgV@YQy=C9(rp*lK^)6q>&hja2*VCUo8zr#u~D}WxR z^OejI=5nUvFeE)a*zRS^g6Cs!Gf13r>9E**i8VhMyvz{)co1_dFhdv zcy5@kWl!_}koVS6QTEZ>uObF0pmYc*-6$!IAl==d zbPp*Ff)dg(baxI&cPZVCv@{GoFm${(K7~)b@B3TluXD~?F4uqv!wqwP_rCULUzY;9 z(2t&6+cK=#FJT}vhXQ#AE>8TD3)o_;Y5A9xDWw{x-GjTkhke1XKe~{rJ25?}EOG!p zoaq%0qp+sDA6HjIg+>X?6Ve@Z!mi2H6s?S|a(cWp)zZk#-j{;+90c8C{P!llyvE4X z?D_N-&i^huSgU+tHw1tXas>>w+~Hqa_rF8vc$4$B?!Sv>G+DNcqDykjTsC_wl&NYq z_BU*`?KHVT2$)$Z)~Y-iV0f_6aCHxmvlacn8gJz(y97{cXlr~R;(Cue84+k>9S<&Z z?iB|Xqp*RY!dDn$)Q>yM=&w`?HH8zT%t{z&9U9R?6kY)=<)d`r;P#T`DF2<~3lfsA z_F+{6tWLFIUgi^ZqbFo&Ngfpp;!$)#ZXWGwiE&3)M@_s+I`CN_PX?~2sILh$5fs6VYgCVj9@WV!WjLhiAi8A>1lmzV}=0Q!U!kJSsiV}8PD z_LBBnSlN}4@5Rh7-K1J27DsNj*YdZ4DjJfk<}|DLPQr1v>=Z?6YqeKjE>K;oS}m0d zAK?C^Mh~HHicJcqS>RzZK!Bi`%>H3kT|K9UEx4fqy4gF&Lcw=axf!~3+dv3+F32o8 zg-E@#${tnY}=P;Um#G~J9WNLzU zcDVJ`^W)Xq^%9xK-*e-GMQBp#lp)%wk)3Id(*<8ue;JXW!}5z%_%#>(y;SA(mr!~h z1@DukKOBjt9?M-HRcsX*9&+BopA|Oo1rn)jNf$Wtlpwc@soa&Qn_Cx-*7nkK(zg>` zj;^sxI-{M2MD>^IqH3hc+-}pKl{<37Z?VY;k^JEW@Rtmm;e-GMsdhKc|7^urW{$W) zaiV1X+1L2(3g2C_(71MX;zAvxM^T7xReu~Hx?Z^A>5Z&AJo&-Pc|~D^R_)s-({!TJ zwDYcOC)L*>#YG#9+TUbD-)mc?BX_yHlxp54WVGVAK*xBB;eZW7Rf>V7fumAb9! zaT%kXg2Zp;y`NU4a%2GzJ!=#YzR!4t=!7j$`fUC>i-AK~Gz9Ufl0Pzm^$DI1HO;bE z*z>K3xPs2g%ot3w-eX`g#0Zee_4Frla$z)z!Wk@1OJq+4851PrJ&N-rO*tJd&E;!x zw;(TBlzhCkm<%G86;K=8&K?O=GVmw{P6J{jOEggW(tTA>?eI%owAvfGOQ&7Juf|sS zCLh~Ev2LooDqmB+wz>wtv~%qVjsmN&8Vd6-T@}AN`P;3hhGr0MkU%mkFzVM?m7)>0 zxFGiYb?yb6c%i~berfRDSuI6Z6%T#*K}$3Zb^+^MS5ut^RZqhn@RI-|`lcudK{33a zUbXg=avAuohR|V+k8(tkZ(|ws(rv#dPvo=}_h|FjKREP>HX1u_t&?s=1lq1df1~zX za!PujUL;;71Gb8G8e4~wfi(F|Gwvf10X z10Gb^?&w|=LGk&x6=BNsv3I@Bf3y+LB<#F@z(WjmjaUgJ{(S4`H0%q90p)hNf~7_{ zWv`nw6I1q9Q~mo%$YP7wkKzXzEBh2!=^^Fio02#5$KQ&CT^4{w)OJedGX5|G7M&M&%4vdnbMkkx;_Ya$(~_Y3X)^PA!}0OlmT%P2o~Z&skc zJr0q63uo8KInL2}W&{jVZCHB*0MG$1hKmokp`qcG!*-w$A>+-HMp8Papb$f8I+dbgqcs%Le;jCQwa_fL^h0y~Z=quq~s^o%zd;^~3(( zAzH?@KOouwjsL$xwAh5sb%QwNrrZIK@J7zBCVbUOv!7F}R0t7h|LsC&=m1m*z;SiV z^zUbpf}ffR2FSI)D#rTj=mW;j5N^?c2?@&%1{Q@^B5g+OdbYy!wHIlL*|MQc|wg!i`TCWnqQe7J8Ess z&!>XzG3M8oKhL^I>1t71(F&@g$D61teYXs=kJ-Ic1ln|@h6Z=xKc>bA}Z|yGU1Szlx4s3dBHZC%;Z}2RpK0#%IDYfyO%yr!eeWhj()K51OIAU5q3Nt z8?TD#k%Hgn0Vm9(W~rs}9m2C!o{Q}v3*M_6sdINu81st93QqBWG&zCPuR>wu3Wodh z;R$?hE119lEpf>`3pGxCP zAv(?ieg*Au+zm-TPtqffYyVX2K9W49Xk;Ywpj;sxA-n#A6wxGA-v74 z0jaMXk3W>ao_?zBsws208V_O%B4A~PWl5IPu>Qyedw=vj{05YEKaP3{cIQ9gO-_#V zGxAMOPro!je)ikS%t3p1uZ|~BMr_YD8lm5IVL~P3@JsCS-`{8CZB6*mDoosI2OStq zoe}LSz156a zz2~xO2EzQ{9KcApImTL$e&zXCK`dC_7SQXv(rg%~XJ#V3$~`TIvmzVu&w`t+e;-UL zu`8)*wY>!ve*CB01!K*hf8P{X87g*rd_I!kaeCrne!N9lI$32dDao=;6`Q44BrV8d zHYp!OoDOjdA><2W%Isb&XWPOY-ZCEflKYNFBgg8A-3}~k`*h{%bfMHBz>0b^EpEeB z^=i@K0TA~y&g!^F{s{jCCctL377=(fs~Qy7IIbfk0F4cYtBo#yV{^>OpuQ$bV_M{f zCt+=x<6!vrQU{zU2Av#(1I44}Ac80AgMiM?2BuTPKLdLgue*a?WWWbHMOr56$iK7$ zz};G&`e%KJxg!r$HJdnK|3UC_@8HcFd(tgWlwMLf29crZ2%@SuBovQbNjtVuLm8>E zpcd38`K{~XNI*~ij=~^Db5&*Ah7`<1>pofUa9_PFSOGy?D^`9P?6t>>dR|GaOHbXU z1c*N9M>T1HSl>WslpgZRe^k~(5uGGq5?6wd52XtJo(4)Ua7_Lmr1bGb6ph*N9ilai1G zM~28SXja=y#2f;3!t+8Ga*GSwQuoigquQ{81L2MW?~WABy(CUI0a{{OTJOS6@{-)0 z064yBWlesAS}Hi1aG-qus_blGbHeR~1L&#phogeTrBTi+_O8lwS!A~BPB{Vf$S<%ab>$q1iCPt9YYW_0%T|Rm{wJL^w`q-I_x>yg6!n%W^hQQTYQO zrSa?+cMQS+7X9r&MEjK+3N^Dt6djHBvQ~Ph_x(MwQb$A9t_7ooZa*E+#$*_WG%hzM@V{$j zXb^_~eN?BN*Dc)zG~zZdqy^f5xJ^lSF&+NYsb`M5vF7HzSBVbSuMXp1xg25&qq^vH zU#-r$#c*AbN+zSH@sVFbCAtJ&v9V!i_a(6ST+^TOr`jT(=iY%#X0a;jr)Ei@!|j5| zem!CL57)>hR{GFomHgu)g71P3^&w|#=m$Rjd(eQaAe}Msl~M8P?mzjY-%6WMZs3j@ zxY)5zG4inyc!haimyMnKSHj{q7hOfO|3r_K1Pm0tVUgwDX;s`>z@Ur`$;l8dup4;N zm3MHjO$s-6>T`Kce0=^OWx5DOBD=$psbXFrE8Ysc0v6n7J$Zm8TO-MVc-O8Kz^Lb= zQTR+VvpU?)W@dA=AwK^1PwxTFBAzLhmqZdex8zZJG`wd#UQE7C9o>w_NUd2X&^MGMGUL+f3yuExx*{D&;aD2;4n6bE*zb3unl4EV)*DiLa!*|7rOpnEh5#R3LyCN>_M(9>PsP=B-W!gXh-GhoTK^}@b<<<@ACe~CWN z9l3&E`uQLhqUo=n*xyieABIX% zp{3>8RCfNm&rb6TcOj|~-vI|uExF=1i@FX;oP^n6W*lP)A14AcCy^DZLYJvB%WMc% zCG}A`;B?mBzRxRo;f!K+;f&~y$>3u^i)|8JQ*FEY@hYDXz2$Y}H`mpsZPhZz24!?2 z?t|nc>A^H|yJ?wf+_TFU_3s^)Vaf|uT>!MatUH+28kmd2sEsfvK4`)SIXA^yG7v9v z*z!h3!wizA@E8A#&FNrWlIZJ>Gb<(;BTq&kkjv37CoNA7LxR=r*5gz$z+p zRZ^78g$tmxIKE2ggrkW_S}=|b25!BE|(?l_DldE3Jrer`u-jFsZz0q z*|_uVS-_*D<=d~@pFo|!`x})@Asn;X`{@%&&shwcUf#UuqCZ|Di5AGt z8B~WK*pM|C>kgx<6fG<$F8n+9oCt8w`8Mh5k0FuyiVDa{4gAz)C3?GG-2IfJ-2ddB z;~)J8_q<$4kC*kbz-=-K9WE>g!33czgd#Aq*|Z#SHPveK46oa5!W|9XtuX@M?K zsnBFAv%6~s6g#xcvTECfQ0h|2E+Sc>z3VLwn#+3L874m>pPb?0OMEMGHc^Sa*y`OJt z_V{8qLHS^F1O+|hphEsuk$SxVZG;S!&?>MkI5^fw%9+Rd8Rjst8UQc*eSuyzUfe&h z&NH@I!EW~8y-xL1r2Z*OWIV4B_1Re#it$*Px4V!?`&(FH=hW>7XTCO%r<=jf`1e_rt-er=FcaP#7N9)LLXOsU9 zV@q>+NOK3@IvPCLLA)_Wlsw_Vl#nStwqnu7%D^tL$*~xQZl$oOH3e~kFM@78~YOjBP$zx}CciK!u zjKhIzyVQ0^N8Yf%UK$y)WM&5FSQ3D}^FUGv3L4ZyBLCriC!a-;Hen#+iO!oAEN|7g@Z(=UEMd#~caUV5b^am2q z`47L$0We)$%8I{J*PiVZyIV(d+!c>(Z=;vm9VI2Imgy%sz1+XqTK)ETJrI~N23x$s zN)NWy{Rf^eYLCdPV?yM@GAb2<@%OKC8{JAC!318n4$U}gHu*(+_!ZCjrJ?@t4KWAT z9v6$?Z@+xMe9(>m|B4n^mQJ)^eFggUMss5#`R8cz=L26PV0rxo4I9~ie#9@I$VwRy zzBo7ftY`muasKr^s2Bhs{7%vQ|4wNl`%!>69bvWr6sbb|XMNvc?ZU1{-_!-oSEY;6 zUw>?CE7zZO&qgcMto5Ra%uAOkdcAvi_!VG!UIIhmPrf?C7E_fdE#1B9%`-fA4lnBO zM`Q&v>wnKDU~{CSQO*hEasZ55{!gHA@7(_s6jpI>Kn0R>1%c9VKECKfYg(RSE^1N1 z>?^6@97D=tR#WIE7aAUcIzwj9P>lU!c|SVE^7hb??BKV#J2KWwXd84)SK zCT(P%-#2?se{0w42E6wVCit%dU1r47Z4x94*Prn+D7jhmb0gi zZ$$oAI5<^`*27@W>3;p0{kYpopSG7eYqH8o?)weXQAbo@aro=n(Y+1#>&0SVWlXof zA@Pe+<<(q$NBlUN**K)FJwQOVMqf=b9^K0a9{=_mh;O1`zaw;T<$PzR)~s0h?{d?z>0<0Hv-JV%%#={?dS+upAd4X8ZCF5b zPkR=9W2*CLobkrole-TwzcyUBe9>PQ(5%6XOL6l&zdQIcg_cI8z<)SfMt3=H846m| zs?mzipBfX-wg@wwX*=lKM?cYnXrc`6G2TvJl$?TwM92&XHw+bZ@Cu1<&W zGwM_au9}UvdQ~T{uK-pX&57St(>J^&aO`>Wjg<9!)>1X+XIY?MzkQ1gtje9tk6@c0 zFCw@|vlMRT`REJ$WLCFgwZ?t5s`U?Q^H<3I6IAju@j-BUZUt{Rcq(i>Xm}mA+oxS33R-nW{)haef6F z+1;zr_O+B{;#OL_78ki6@Vx9!D_Z{=6=>e&(%rR8v)Ov4p!7Z9KUw3d`=mmbyU+E) zo`3vZTTFd|@C|qEweT1fnw0;wv?!;oOBO`Lar@)~xusa%f2?^YcxV}2H~kN35lz7S zz~!;-q7ZA8qu=_l*<)lZ27fx}j)U2HThN09E)Q&AVl|ehc9XaM+2hIt0VfvlM`r6# zqVf)9Kx+<WsJe8C^;9WHR5@+OzkBz$@lv+ic~G zAQD2@wm|-Hpmv-;p#1!x8x4EhmiH79htY?Ik@`M3a`oA?(pYBw*)=^3%(TPU4f2$7 zZ_8xTJ>@PVzTU=b1kHXxqwElaP`=6CcrQ4<>t13e{^uymSAEpi3VEV_#<8{_Kz*Ug@S;b!~8Dx-o)IE^e|h zUj2|)KIttk^0$Oo<`rh6E`K7>fxx=_@?LASMEPh%LAp4(fjtDgIyIBpgS8Lxju0twi-H9`L(u`_gVv@Sx%SxVntfn zc`}MDMiXB-Nsf0P!$@WNo>tp$4t(jxc~>Ou{v|BB{Dc7GMYp4~OFrdNX1r znkK8a$>Tq?%Axk8Kn?Ndul}i3Mt%81tE^LIacgu>ETq*g!0d`H@-rJ>(7eD_G%L&i zCpDb7DpdtM(C>xja(~lr`|8wPPB80D)8bS0Q>uhLPOl-eK-NKNba;4S60y&G`Iehz*;$4%gx5gIQ>-BU8u3um2tS5eE zE=)dK3#>KdFxpNfM>DEhAmUssRO_3VIC_yQE~j>peo&loxSOW}n0SHOMbB&YR(-wE zY*04X2a81oSNcR{jkKR(N4@r(Df7;L`hwg}aPtnWh>64eYu6#V_e2!|jNddH>gz~t z6JA*21K@IAtLu5^(}($^%My))*r@<~tI`Ssm;Ekn_I1ZQ>OXn-{n`v60l8SNRM_I! z!BMTY>f$6CZk%>W#I6>PO8CSmk=7rQqk$%9t~vai4ZMNPCAP5Np(Zet`K8K=Fl(i# z%ru`!`9Qe8vSxmBLV=Ul1cddCRRUY4W|H-_&DZ|h7xzd%)~FzOO&V|L52MGin5Ggt z6LPwyxW_yLm?F5Z+$bdJV-qYULv#{+N=mw*e zJT-^V-f@!iS9V}^f!Rn(tjyAISAXVxPY2onMB-PHu|1i`EH!wCY#;eyZOYk1UMaiB zOc>~9owM%sK~*k_QRT_p)ng+LlZG|qWm-Wn+mwU(u*X^4ciLx7nJKu`INuk>wcQJShi05_N%X+)#(UQ-C%vRGDVzz{e>l{Cy`Hq8V97i z;JAVO1CT-2zE`vQ1mdR5!2HiNk&5NYjIHZNs_@=#&3bk!%#HgG50Wk{6wE3xd}P+& zeSvZZEi%Mm`UO-n7LY;B#&Z{V%S&O}dPy`nZvMbFyto|ol^C@?dHoRV-7%V06MHs^ z>}xd|9eJY96m{%(6?h*B1AA-mbD(lmG3s~m@rKX3aoIbiB6ug7x+T!v9AKVaTde?N zxYPQpbY6@9Nb^Hy17|zPvfOaC(XK$e?y2cnl&OuORLLVPcwz0Pt^mFF# zS8^g-|4lD_-MS4PK;ckaL$c=3*m}L$r;sf}{@6f5*vn@R`Dtu=)wacO4dDt^>nhuF zgEGle>kQBaoycQMzI3J7+IXQK8iL~!!qX6Bh>JmpQ)N3%6 zAK}#DT0pLGT~A7(cHg|+h9`e$L8dSFPzcu!Zo!c<&su=OC)E4bRijr5IfQQ7LkJrh z5K=SBbzsSMRgKYKpbRz-<1|fiaot1P@HJi*P?@Z*b$PoVHD?#^NptXtin4WdQ_3KW zI+ap?`op2th*1JoEQZ?&8vCJ!R!Z)j=po8S2A?W?ju?oI<6yqc==U zK#0+#oE!XMnmIhmqd`@TB&3A^)Ygh-apBLmO`|emS%?-=cfBQ4@p5nsQfN>9>)Prw zqTddy#djrDOV^Zt00Bv)7Fm9)F^vX6w%{-PNv8-u^}v4_MO+TFZvW{L>ca{wgm$85 z%2kQV&9DPTp+13sGYSnIz;M`U(*;a5Yow}6um-799!e*27Gi}R>f;J1wYgp%u2di2flQ^-nj0&A`YqCX6)=h@wEN#1MT`i3 zcCZ$p=41y=_yMazgK`itaAP+(Tw>^kQ$$yZt)Z2sABtAWA6^lsU%RGw${lner;9Rk zW?B7wDk|+fkpp;XRR%WF2w`DQ#KVZv=|c85?sj~*ECT{zePqkG{KmfeEL3BQSpo1tsh1``qsw2?&JL+x;mrXhMTHm*yJkal6`6!W!vw0{PyuWgPM`qYs zXpVsI>1f{Q$accq+8`1`m%8FK@6JT+!-srG&49;hD7CF&+0OT_?F7|zZp7oA=>5&n z#XhjhMHOOVHs8^t(;?K_u*=Q;sdL|Gw9AFvVfVF1rljNSr^CP&;H2dugGX`M%$Bj9 z`MP_Xm;ZfXp1%a$N|Bo8qrZ}Y>pu)W$x%(u!N4~)9(A6RBN}hN*KW6@z9NyCLzO!|2m!DHrqH9mTUANM6(ZPY;s`iV<*F=i` zP3m6~`rj-uZXJ0}ESKsutF0!*?5Rbw)}O>gjKIQ>x0 z-jDMLzl?ro?y3-SXaun21qN)qg6pyQ)BA;Y#N!lTmtx5hk8` z^IDLF`aC#Z=?(!}ipM}8S5|*Qj>=(Q$`dM`LR~K)@RW%-zT({HH-iTeOgh;IXNNn@ zz-E~By1cR1;awQcoAwWdIhw^Z(#gD^BlDy5b{c-_U4Ze!D!-$X)4b~%M_~L5^!fS! ziQIK#YoAI;M|*P;juZ8p-o;rTd=E}QpeM1i%$sf`O)EpusOwL}az zJF~~-bA>D+vJPO6Zfq!PS&!-U#P@wEtCB=YDSX!G=l$G_?4Vh2zKYJvm* zQV17#^;ZrM@MxWc-y`CZ;-c5iDPQg3^B1O>Z}0!*r|+48P-?^w4%m^!efs8$sw#6V zahHpfloZk*fJ4iGrDU~#u&!XZH2=vuV%ZPb?(*=ge@OOjwf`sY?g69Zk|fYXESo{$ zwEUIDbd|O#kE%*ra9oa;6@AOxoQ}4<(;Jd!(~;x%2i6eY83;;c^%k zStXtNZUy4Kd$G1h+fx?x!Xd$S<-rOR{Y3E<5kj_KD$N5jm`^ zl56^tGpxTEhI2U(Ihia_nQ=rV0=wX1R*zP!x_!PB%fQ7mDmG^T{dd(aBjZ?C=)H1O=hhi-(Q@fkJeHuEhk=+9on35V)` z{4=oYMs$K-=r&EmR+{ss`@*MWiPn*;AffZM;{Clz~0Px$}0w~)jQKqPDh}=XE#0g zBIY7@*VU-konB6w1c(8YCUPlDU+RsQalbV9=z6|>Hf9SQa~D;wFxG<>Y*g4!QJBRM zz0}Cn#X32>LT|7uql_UZ|)OE!cp8WL!xsAwF4CD))##b=P0Ds{pO;Nspa%k8H z{~1alO6h6p#s_ZlI?Bi?%NefK?=Cs))J_+nq^OLfQkM6RrpnBka2rkF+zG-OB6t@BRndK zulv7zeOF?%W58_6DiW+&vJ|9bh(Al-jU8iRca+mq+F-=9Wd+7(&C5n>aOIe#VPR#oI@56)$@cWb0_6C zjh~xUL}UG-`|^bEg1X;*1Nu z#0=BO`}*H)uY&2e$I3$zIGrSea0@BPC8DCP9&MUbk8nf7TK5eZBNMumlaCyI8ECGa z#Wq?gd+j^^5~8<_G%$DW;Dk2COM4;pzw2 zpduOk$C$=EQI3?>M4yKVc^oo>BS^C1pM-FRv`VZ};+EGyAtAkfvV=ceFTIT&@A3r90sKj6D zUhiXQq0&8>S);wYo&PMoohP2kNRCwUehk|nd`zR|m^w|Q655nQq;v*EMR&s3apKBC z_>RRn?V-<$q+HaXV};pX-)bSZ`_blth*|SQuAr<+jmHZL%(_)N$(}IdsZY(P&EJw< z++G6$*u*Ot>Lqu_Fs;s}NG&*VT?k){dP4W~w8QS3jZwTX4QRQUEt&h+Dtl+Gk2N?? z^yjf9C9;Eh6pVpu-yf8JBMIgOf-`0&s;m1KLwocmgkSo_N{jVlVV%BK&I zA4PF6&Tq1-#g{EXOV5-r%Fy$ zCJ!e{>?2%{E>wWXoT@>LOucb78wDx61fJSPRz{EVBHbcscEYvI@2RyMMgUQSZhKLq zsiMP4l~-A$^jP`6FtN$7wsPZ97RL&-a(6dv%=LhE!~MK3@Y=gGrC_c~THDDfZY?F7 z=w%$e2HXDnJHqpxc=pEgl)iHFV(UG#d?yKF zw{EjB>eRtq=~~+;KwfGVOEb~FIyWA;{FLZtR_X6o^qCp*z#L|3qnHOXWUd}jcvu~L z{@2AG(&Cp1h#!(!f|?EK5)62JiB*5U4E<#0h)u69K+(XWbeO-cLrM|!UKNni?PoX1 z7Zx{Y`yW9q2tPZ?LuS@jS1FwY_hYGz@@KVc_Xp;WZzWs!1jD}YwEv!`17&4}j}$cK zS4LDl$Jhmf9XznWiJ0w3jYH&D-ud+MHF1qSrAV>z}UP zAXZ3}&+HO^Qm&jkOHp^8R_(yoxXI+YCR%=IgjAg&>=#9&G?}phGUx(FZoC|a^lxp| z9W{#&>O?`Jk;jzf%~I9J3`Z5tp+EE#m$1628TgWT-Ne1Tt`hH6ev0E&1BKC~Tqs^; zXPU>>8i4UlYV_K5V6%D0)IRpR%oG%31!8ek&v!miMfUu;lYUFU$K{$h0hQ#{)m26Uw#&ED zcx?I^YAT8{tOQtunO^4OZ=$C%6&T^TeEuVHL{UteJe>LI7MDgN_{qTfK*YO`B5~_R zwf3XUQyPFb5J7KQMIx7dG(}LH#4Pto4`+-sp%mex5rt7GCp2YCXn7z-TMzH#{1x-o z#tT+|<<(S?!OOWLoj87j=wP0a$(ZLuX-)(i5BdT78sG*l7;+1uoaqCmQ4>Zb`fjxI z{wdM$@Dx-k8geJ>oA3IqO?tJfVNP3a^0@|iIDO6-rp1-25p}jO61&;tPyukH@s8bI zc*Tl-P@?Wq_DxS~KLE&cO=mm`V{c1;xSFzk>Q@l!jGc+G2Gi|@xi_$yo`Tbe# z0mU{oPD-OmV?R_f{BniUi2MEU8femBuj93Fp!26l$-iE5b<+HIhexiEDgdkA_@=ou z@_qEDI|W{@oGICjvmTW0ODVkb0gU5uaw$^rWuY>cwZhvpE+JqlJU*yZUr*~8ckojl zx;shR^g8bF*z2I1m0q6DFp8vyQ-w-k?I&L&t*2sCwS#G8ANn@005R_Gl zXsQzQa>J;h+$PsPQDn(hlY{wFE4d-A|_us({oywk8atjM$Q~UXIHM*+aP+1(xWVEHu`K^BXyY##gS;hdE(#1<4H*dEU&;fi z0k*S4SRg;`bog?#c5NayU*z_no=6YoR|pt#In)nVPCdU^k-KlTF=F!Udi({iBj2ez zi{c=KJPDz|oUmHS7h4_4y~9>CQyAhh=(WAdksVww_Z2 zwJTId;w%pWXjl?(xYNcHS*wv1_^}!rR#T8WsTPWLrq`eEuhKs=>Q5F647qjE#G z3^K+^cx+ef!pRboXAe3QT;dClCTv$n{1neNx`CTF>~eFoIZ?0L*0*aNgECQD{5rJy zbLWHIGg`c7jN)04`ka>$)mW39z*}NjiS-%JwW?4iC-0J>CFm9rujJsK#_Jk#r8TZ6 zUPy@)xd{I~b(jdtT#^I+}gd+qg*#!me7zNvV{ztRi;{n0MMZG?2~0l#U4|H#h$ zvwMDp^Xvt5+V}q}pRuv5##bCGUo={p$6!%rW2uypx7<*pT9*}%i!w|47|wZ5lonL59%Gp zx-$IMKGvd1~O106aHQ z=}`o%67u_SUnErbsZD-)&L|4Jn)P=Yc+jy3>Fb@jp96Vj?Bg5~@tg(~*XK!v`Gu5? z?q(moF}#NI02#|+8+VXy!Zb*w^)BI2%_BeR%lnBc}{R2N9d z^!8ba*kA2eEu8{VfS0JI?^k3b{uE!b){|8Njo|9RTMuYe?VVL9n9qBxyZi#qF5lyF~JSXYv`YGn2FwW z`Ye-U=b?o3;DNYn%<9b>E^y^zRXE_Kai>he_ud z6eS}mK}j=5xqP-?5<^CAa5u9Ort?J zwP5b}BallXNVGGc1em@8FNae><+}1WfQ%Wobe=Zfnqd|_E5Ck8V{^Rw+6mN4Gg09d zz)@tAY09*g`_9zF|1F36Ld+XU{Par;S6(5fwFvvi5Q){Kl6Q-oC+FQV1>?2q?cuEw z5+U1e%d;-s-GFz|tJ=_SpUL&llCM*+y=@K;?~`-AUt@r1RAA{eZo*cRJvzW9Pmo~7 zg9YVfCMPkHvOtD4M#APogwZ{opMA6?53%i@e?~T|b58Do2h7Ce!Z5tPy1 zlr{7)F>dOql)J-F_pp9n-8u|jnM~;cBed~JLNPuJ6f!i4#4BoVl%Z8o+)2Itjhu{3 zHJFeuZ&omUbYuT3A<0*ZzjB-P?*irS0lUH}XJ%upeDSB%Gcwn>ihj$Mna6Wex1|zo zQLi?$KE!OiTwsyv?T812Z8L-OQTxgfD(U(etMIJ;&P;^3s9q7NMFJcgvm{ROQbbEQ zWet5iY8o`xH4%7yG?EkFXmd&$nl@tt^!Err5*N-v8RoeqV+5yw+Uaq2fwM7=gjGcs z+L(gemlzJ1&2#~J=ji<{FQwCd`FukyJos0~jq(bv;jErWKyU2qZ!`z%nvR77C}f;| zX;ys~Db*{{ipyd#@%)l(=_Cu6T>CjWqG~~#ZkPNqf7CQ3Djhj`Gr1OLDRJm40~j3! zrdH#n3u`!x0gv4|yi;VA59*Bso!fSY@_K{=l67}zc5-RdyV~@fm#R){KZ=%Vg)ibY zK#OgX;kzV4ix9;W`<8vZ-aY83skjVQXi%Mv(BaM;Es$0e+FZHsAsPwqi~~syr3uvJ^bn7q~wdK2_s=KFaozQ|jl-#~NyS}qt9J&~96Rv(m`AhrI*9Lw# zBnRxwI~&6qW-EhF(g8n1mJPcVg<-)&g=EAc_2t`(j-rq|P`a(5%!l$d`gr-(my6x- z7_*h?(lTIGiQ2XnqdZ$ka=3v;x1WA zE=wFh6yM&=c?k%N5PKnD5H6btBV$(a^x6oId6pUp-`do!>WRzIR@5>rc4yP=Z|LAXc%=@RBkvVF(*`<%G?9BL#j%z< zZzCAQK=>*n8J>_OrLKBJ&!t4m_v|%|>MPx)w-d$tpcCtF4lrm&tYMi+7|A?zF7v0> zniib}cvJk8T7l_gIxkBOFTJCSZB%<950^nHe(OnJEvW09qnkcDrrk2prB+ddN1cU@ zO`1ZQh?dLYqW?TzEhK8xv!^REFVBUSTF{5j01Bpb$U%lFv!CG5;buY410|E;d2fWZ zu?gSIRDwbRLvVj8t@%k1a@_^vll>iwmAUUI(DslLSKv(9_Z0dMp%sOKvg7;8=OpaP z3q3+|vR9R^d!yly4JXg+jO3`2_+2r2 z1$zz&E-S0XQ@h<67(e`eFMcYA%}VGTdutIPl!~4P>aSV*UMY@mTHgJ{;N!v8KG?(? zPB8wtl!cEKe{`^Z!c6$0T>y0XF}2>M54m0XsSIbLmU0fey(ASs7sWw`f4}j!g72nC z8 z6Spv{yWW-DWY=+{LJBEMz5Zz1#xM&^cy&(hT$X`a2u;bEJvgZ0e392(@^rX8aoJkw zg3AFKcs!VRsyWp=$_Xv0IkGfj7)_T1mL5S+`|}oSX4dspokCU<&STYv^9!4Kfpf*i zUu9vWM~}jR!lN^sT{+fciczhiNiw}wU_%WZuc|W2;A94b0tN@{03%#mOWu9ONnnR6+V)1I*dI?CB5C^3WuJ z)-Zs@a$52fzGGT4>^a>QL=^RKl~I&$RAg7Fxr~*#Ncr|y-%#ypgrVsox08UushREa zASznCnW41K+o)R{j>e71u49$T`O`dD8EtpU3;OeOTnaQ+Q^`CBYrAGi6?78mRYo4D zT{I6E`PS!bUj-uJ_RAm1N(T9Gr}65a!aj``I2B_szOZRQgx5^w@)|$ZC8tIwDnv+O zhwMaKOtHr|8SkZ$7qk~pmi{%n`S9Tma0}cc)*R)oa8o{wPm2l!`oA9*5xHpDPH#t; zmLUxr7?dHM(K=TucL_DR>o~f5h-K+K=xm}YR?;0Z-Di9#c60VW`}>0d!wjgFbrhRf zpBd9)xf7GS@*_8OWv*bZ=!~M1I*5Rc`M>1o9E0bVgro0aho>30r465p zCHPug_8n^FE?$?~z{n(`cOi2hl1>TtR06q~3x9$(H0?8T>-9~*+INt~dPgLb)}DOf z98l4?i(;ShMQ~La9m4zSq_oEiRFsM~lrEPYXA6`CuXUL+iQeo*-khcT~7k8Q)& zh0EEE=;FCVeIl=|IAl+~ClG(+s&G8r$QPC9xi>PlvRRr>tcj;eUG;ZFZXnZ%j4>#l z#QULltXYvxW8oOC1xOgXq24uUL~i&P+GL#FUYjdddevde$ZZj|=jZ`9lCL}zT&dff zDDB3$TBo2K?U>)zIu&1)o{`cnbayG4d&Z1BiteXfy+Df9;i7msyBCe*-*9wEqgL%4 zyQES&U7&iHlt1Bo;pBX3c;P;1w%cIHJX6MNs7542nxHtWxj{eB>C%zV#D1fZG)Mdy z>mDQ(hV=gWW;DV)J1{4?TOm3c!GaQ4VKGi7W^h3K0h=x%eT*47;+i5CbpU4(oUEtH zvE!(Ny+G@UPHbrL&kh-Ff4&E&~JX@&qpA zNc)@fH1#~2N+=QbZE+yd&(b!P1_mu-4Vo|*%; z;a2yv85N40+v^p2_eB&9yAC3YxX#+R&hf@nDX9c5r5-SP-axk`~vq;Gsh zD7`7o)K0@yS76^$cIPy^%}HU)JnE07Mn{z^%SDGuo1^Q>joN~&SQf7Q@-o(3hUX7h zyGjG7Wjub!7oD6oi{xjPFx|zE(rdfX=$8I^J9w~`>C1Ws|f1cYI2so`o6)MQD{XNqs2n=ue1I2RyW`G=P zsH5NIq%Yv)%%{?FW=*Ui5^o^aM$1jo8Im&%6}*OlVz_(UTu#`Yks_pf0nQPv_``5| z&VGS7YVQS^_^2D%0iA3`Zgs}ylNWt9)KSNZuVvB@JHcYA zzM9(D1w%ushP6r|yIv(8(;7UAE|Pu~IFf$v)XU#7^c#`39Kp%8NBqJZx_Jc@n$Xw< z*oO|@1g{++?QaYWqcE*f1x}LB)jHOQG|5sk6mD8_b*@i`ull^v+C*5QwWq?UkCVRT zrr9FG#&j>14`Llsg^R`>gC8Q^MQDWP!hHuHqEW```?!+RkZgl-}wT8)?hzuF^Q~B~b^nu&{)mXW+kAl+WSfoO|g> ziEYNn^HACN`>)N`AGGy*Smh1ne7O;%2?Z@UZQW=kllAQZvROrwr0GbgJ^rD8aJ6dsV@ zU*Uo)_^)U~BjW%B#VsFVr(Np0It)q>g|eEnheQws&nbn{R1B8q2UvwFOFwpb7u}=u zk;`0XS9-8@Fs{EA&YnG#s;hl+!V4shK23aKDZ0d*tsOYKV^(`GWqHIk>QYbGC6upi zSyo}e?!o3pIHwI3tll_cS`XeS^z0fa$r@TY*xC`v&5onM69{&vIzU}c4D16&MI8Xf zWOYVSzdIdL3fA^>1ehl?uYDy*=gUd~JwCwAqwoV;q^}1t> zo+zKN&TQbg?I(!xd!3=~0o2NxBTat5lXa$YM5o9>)5E~|q=KjgJg`6NV@)}O)_%QE zyfa}4Ro}lInm3Rx2T)8O5Tv`uq7+qPZd&%wcbeEqueSCz3QXe0*Jt%pk!j;VL@|&v zAco|Jz7)?M_1RGc>ZmiluMXKR;%<^bvus-`kGdDWmC;b!wt-ju)^RkDq1T1gh(&Bs zn__BRv4ThnVvF<9NSz^6xPQyAEG(OTS?oCUTZIl(rx>#Ssop1e>Cw2Kje8t2CZh`feoe6@2OUCtM zTIl74u5}x6JaKfnNvm2*SI|?Rq4u)J5aFY(BV#Uv(lL8Vn{UW`+)RSFp&XA* z-cswCBqt}gMnR+x*9#Lx2lwh6oq+Sf1)sTY1RARzczt>9NrtkOM$r9m)r10}{tIxH zg~w;`Zhuo$dgDSDRYKK0Z{E>cM|jAeqm)NXX_k+`P@l0tDgIF-dt9mgxeWC#NK3h6 zXRC=#W5@s~u@utypLd(s9sR{;5;PTYZ$&&4Mz24RpZQYq^!=NdnMg5^e#%IPtlv1n z@#zaNfj;ez&*w;lwU35r_0l<`$FtM%1%Up|FX$7L7L}*DVdoqj!QmgfVg=>8!BU8s zmkz?os{m>nP3!K)2#{+4_LF(m=}Ku(@s_9dp**uusG3?G4&qaMSAp%SY0@{^Lldm< zy)oyh{lVgAQq>zQj+F>SE3r69_nagd$9{-ze|iIMdYb*gg9jH7ijGFNE(=ziXlM{{ z-N=B<+AIpqb`Bu>FvuE@{sO0R=LAiwy%3wl5Xg0}Lq0{}R}CDt3QT*6dt0AF|5P4e zyYO;hnxpMbYHEL%*-_9b*ElJBeZZEq<)g96dM{R@9ZXEUOhiMDm|v<+d8y|&G=h(O zJwBa#voP&Rs;ihtue;Dbms7M?aldCMR%&&|c{8WqJV?gCf!)ht^#`6f&V3&!Tzm<` z!C&H_{SzU_BaO-Z6Pu2k&nLxx+T?$H;pbPA+ysOQs7%&svY(9O$D{t!D~B6$_Xn6r zNq+wJzd^|VdqugwchG-(^*@+L-*G4AQR1K0Fy-IuPP8G5`?QI6J%wHkpWizeaj8oas2g=K2 z?Oss5w#sGWP7k0PLG&#UY4qgee?eU4E4(QnP5QkV;Ipwj$zJWxLlR&D82 zsd%}iYj$G2R=IhB#=vp9`TDIZ(#P@M05=jOQNDPc=p0oPx3O{YTh(4Vdv11VS8#4LjIKs?F%H&*<*hx~oo5iR zG`jm(%18*x4KtEqBaaD{o*-n;URSQPlL4+TQth#!!09~Y^o!*(nthLD4^`3{kiN2) zuJR}NVq%{DUB8YXBC_f+VZjqH7O+qjOV!wl@%LcPPrDP^xb9Kv#LOdVTZ5fRuL$v2qfP=MX=D{K=K)W@|rJNwPp}(l0gha$` z7+Dczi0(GP1cmWdtVp&qyDQmyzz2AEY|*oly|Td$xz|u~-eXT3N?j?rM-QALaN)?P{^3LOE{^{hL~7CN z|5>ZC08PXKaXowOz#*=$R3gF<*qL2>*ezSXX5YnffL88N&J^0Wa~SZc4enASWfpaV zLh5cxyr_Xs&{H$;3*3_4w812C=1}uHmLKfyh^%dFXrII=%q6u9yWO*FF`~QDY04xF z9ngfJS=iVVWK8#8Iqi4QsA)u$bVUnfuEuSLyT(LJj;=T9Luco~qDnhFKCIS7Nu?G; z-p+FX4NNKMGOH!P!Hp6H$w070(WA~lONr5A-=m;%8;pwWKu+fHNB!%~Ab$Fq3#1Fe zaa3I+IiAh&n(qNld7RfO2k$cd@(oG%m53VGYpp|N)@t?PRjZu9slfuwP^|YsN=+bC zq{8CL-zx<8ofV<)1qU2F>$o=Moy}gN_VvI0haAm0p<)@ib;7{R8`xldjaL1{D=c22 z7h+heV(&3aZ{;*mbDqhm@_={pf#ViG47yY!icKn|DrD@(DwU07X&49xh*wdyNoV=S zqyAq|yRBheqy)3COSTNw&AB41reFm%f>hnQfd(38Bzn@xI-20iC0TQ((0-S%CEmOH zvmc&)%k>S;^(xAFobupVqeahn$K|V6S;U=YqFCVt0_d+iKVUoQMMz-!>vY{W6$vKw zv~WtA1aNnN86h;gQEJID%Iotz zym)3$l^&1a$#$M(&(@&mHEp^cBPOtGUtaUFFTs z2EUkgly|ae*E9XJRmx>id}q+8_-Q{qvdZ%AU^C_*NN>CB_@i1cXLUsha!+R>ToNnh zboJ}NJv_v=iuH+}iD@QHL{=!xbH^iDy0-o49}J~?F$jLpcIP_<6cKq=CMs@%#?4ZN z7;p4$mlEy=Z74;2nF-=@O`_m8<B=ww*uQculf+vYTA%#e8LmBhQ<)aG$wrhXM z9e3xj6pUNvw@bDb)>pJsE|@I}gJr+jRFEvChC0|S_{?m~>$z?WW->;Z2c;cfUop#Lm!oKyjPi6KK)x2sp{-ZcVp1hjRkblr z%q8zrMPU2@pNk;fh5p>t7fSTCFQDybD>Pbo=&t>(@)CzB4F6 zk>vSVy<>0Ih{@Gx0S@I&Lx~O4@pcw)5Vo32=xe@+WY(HJqwfZLr2ZW-s z4x<&f3`+>BG}fo9LuIabqyuCLWeN8M9(E8NqR{qZwwbFJ$eXwoRDLy zwe9p4jq3Y@X9Ee@$>z0O4zt3Gh~>-BxJ5FT5-(y zz^K$x6%h-Z=*P|AyO~mV*zJ^?lSRgpiwPez*3pSU#_5N551t$^vo)FoHs477SN z+UVD(;zlCUM3QwQc(czs{1--<>Zq`yumM^Us@>l#D&0{Eidwr>1#rHNdnBDU-<*F` zQ{}xoXw8-*3hx#Wq+I+jF%n$kienADuq^sBGqNRhu=Sqmk>_zV2&)7%o9jZEG`OkD z?9KR;xy^Y}*1^ULDrj(f1teF^beN^)e_=*O;x#$|DWP*^Z106x_-4Y{6UDm@6(nqn z{!{qOVC{lwW_}Gj-GON?L6j$2dwe0+amz*Y3Q7l;L{;W_jE9Ew$EvF{oC9~ zaHdmcp4DcNj2r8EDaqG`!}xfp)6Caf92pqaLG7glb*wHZb7Z9?tiRrroiZadQg^k0 z4+8etua>}Qj+<`f8p`T~6Z+21;skt`xY$gIRoqv1TO`h6Xv;U$iQ|TTUWpSY^S?`5 zHbo2J#GqA+b6v)*km9=g1`mYgF+?EjlY>i2?g4MEKX96QG_3Xtu@ajtmw#f}CbP9v zviw0DOaj^jxx*zNH};AY-GJM4DU`S|-=1}axj~DRQsf`5t4r z?iAT*mO#59Zx$g?4ssnU!Vho(7injU6;n^=FqWZ8VI9tu#fnpN?bM9(yeX;t5R{F>9BY$!W1WIyv{5glR-RH(c3r9HVauW2i_;1;WRmpiusFr9$GN$kFOMvztND z8mwR760=PF)F*xYSWnH!x0P315w;Sjx^2u3G+1aWFGC1roU6KJTsrG0*~g@_Nr;>VCdV^Yg#+Fnl;<@zfvX8=vk8(DKn(pzipEaw@rnB*fFY)=B@z zgK~y(@-8Efh~8#QlOQf5ujmf<*=WB_NR#L$FSw*@B+B%;U~j?BBl8g6P^k2 zJP~{I2P69TA^!yrFglZ5yxL{5d<*vP|NqnKoC3faX@8ZtKMgaYn(#o%G&u#Ic`K-s zQ*10bKbaYQ=B)MaOi1x6^z>M!ZgAz)Ji@f{csEcDN%f$s(Qr7JnXUDZl2h5vF2sAg zDSQ2W;Hp8fb4;8l$c$RjDvE;S96}Yr+fhe^R!8pLw#HY4xMO~X8UOW;S{+V?hhHr; zsZJmye%|`(4sjtEYi{0zPoZg!^9YHF+vW-zRB>9CmJ2i0r!zb&68h7P!9gJ*F~DK! zDuA}pq7~i4nkZt|dQPiWPkN{p9)4F|QMH&Jqy!wW-UU=0vow(0fG9YU+t@F96ey7J z9|;2tqHtjbh%BYRW7c-OVZx_}U( zLcn~%yDgf1AnGIYFNWp+j_$_f+NE?RrfSR8qcH)QhZY3mBHSZE|Dj7>pdaUrJPcIH z2S5#goI(#BxfS{?BsSp5egc&0KiTCzIw|WXdgKR_9e~(`Cr2V=E8$d%5WhosA_b?; zU3~s|(@X@kQoSgHlyOZFvj3!3zyBE;2ss*Z7ArXJN))|3$_4su%B7h3fO5RkB8GeD zD_Y_^jzK4_jF0pW`Bnz=`{o+oy4ntkGBo=*$%*#pJT>{vTm?}6Es7zhiHTGnMLsNb z`K;n`G{hgpE0lT*pcn7}a$X_8Pux#DYqn7MsPFRSyl!#7qY#gxhGp3|`cAWF`FfHw zXFHLq+e>2*)Cy3AkCFwS+!)m~0#al3y{xW=gceh;qsYWOZPPjfWyiR=f+IHOU$Q0O z*?XjuW;;vF!pEm{oh?jH^>t^Aes=v$ai?#pz}XHNhS3-dS(!$%jAs4O+&PkS=L8v= zw*dc5`*U6lP^!^qA4&0fO0!{Hqcx4k9=BJ~<&tK73~iT6VBqScLy$gaIy>ZkXh}*#y)^c*<~*S@30< z&IFz%ieJQpXDm2QhM!j0m?4QZ8{2P-nmxw~u!)vNx5D4!v$Z4%AFYFfgU*4Pd7h-` zJY9M|NZE+AYM1Z<(HxmgJ*;BHvV1(4>9CR`9e$oLm&4$%PJ zxzL8F@<-a^I)}8UZmMI)G!F|Y-wctQC5gN*OcTewappT;ZZ7wdu%n{Cn~Ng1v+12{ zRz=$m*&^kyByEw42?Ld%V_RmVvTf?7HKKxUY?+;L}fR^VGds_)g01Rlk(Ag&NsMp!*u$#Aj&jgUvv*7vIukI+c zM`dng7^35BUydpTX`Z}?8lU&si9-NMDgx!@uF65lGFFH=8b~ZF2O8(84hgt-oEl^h zf<6Tf1ij-sLDArrlP*N}2IaP#C*=1w^I*N>G;U;`L%Vw~bWx2DtC^5W6S|Od#P;Ow z#3sYJ-6s#O$EbbX{J2JrOrz3asvsygw%O!KxxVsyaM#`-wDie~?az90 zmT{XO>ShK+kr?0YlDcZAl4|>2{|B6mLq?^!9uWBGTP8s(6sN5Hx9$RE39shc9EPRk z;=G)kTF+%9f$>e!BX%drNs*XTWIq0F+ZKTHH2&sezQ@>AEQqsxfN3j z_Sryo8uOuNKHu!pJnVYr-LXmVK>ov1Spw@lE0xN}<>sY#kw=8Xd}vY4Qpya{Xx8>k z_TJ~n`k)*}0z)lE_mz9%K2sU!n(WGkT*1_xJuMEcjGXffYvoB&j-xSGuDP4~$W=w$ z^qq$7nvYOzJQ#gJqRj%FiP4`6HO_Xu&cBG)k1Uq!+Co)~v-S8EO+d!h$(1VV>X7+? zFP4Q+$YQ~u*#K2Q0m~#HIO#hCJdG`f0Z7vepuu!#FQy;4zzq!IX(io0dUlqQU%kpG z?o2q0>NYeAAb7?qu~$m7Mh$Cg-}`b?ofrg45_?6E>xkLlmJGv-3!J#ak=p*?PGp&> zmJ!!5f96Bvs`oJ8MuoBp^zOARotqoUpleq3BCizAJbL2KMK?HbT49ybIlMEeo{t;fE$`I9(R$vTi>F|Dg*JV$N=6|nPn>T2{PI#n^WtS2XfagF{!D-x zGyWztEu|%5i(sz+v5JH#G1D@UylcV*yWw;8YzVixkZBOBq^h$HnQ2Noz>LRYOMV8p z@Ge{7eowt=|6ZTM1wpLfNp1tNmd}BXbY@CFFs|=GmB0SkZ;xh89kqWCX>Od9>1^-# z5&L|9_3k=QgS!6yw>|hrboSp@j|xu&Xcoha?9czY2mE$9;O$KyqT;?R^@Fkf?_r%k z_Uy&0TfEq&7riWf5CF{_KXz^YC7O91IF=R~kM6h(fW?iauJwv6y9|ALj* z{{DW4>NGfZa*}bVZp)p+G_L??JBaNXefyiAWaK9LQkI#?=JBgnA7^^^LQ{041c@AWD@$1*lWIQ|JQ(1nGONI+qQv-aB1rnY|%xZ83zLN?A>-?4}r}-rdufudJU&p{olE$8n?G?gu@+F81?; zf?0vCjm7yuVU%=jYZt*A=;!7v-?{K0S)Q&xNpbHt@pANPeeVPMGql7In> zYN`$ihvv&Xc>H=VkDxj@(f>ialf&v#2&X=FTjS=wM%{df9kg!OV`?1K9Ag@*SzH#v ziP&-3t1lMlm$Y0Z>{b8*XI(~PoAKLKNn_qX*yn`nC$i2Sjf5oAui^;^v1UH>2%y!F zGPWDlsb3q$&B``sbU8XL;v$Ka5ajX|pZiXYRZ+CE(fbBb<`CDkcbQ*V3_VJvObt8{ zxaHXg2{Wuo3#V|039m@C$1L0gmA+5?$^aG@EW260zvtxkRmCFau37!k5=H+-xycI@ zt;v!)8mVHq&2H(TiHx4P7#a1;*G2^y)z_KG{}af6^$lQf=XO|R!@+Bsn&rIsnRYo6>TgL&`vyQG+gY%N1KNc?x$bwd}=V9@C zB4vm>o>l2ph?d2H@%#{Ty!|W^0G)%Pa$1$LuRJPkP)_aQ@JV2?A1^aIJIl=+AnvSc zQ*_nC@326qO|)7~AkW0KSh8eEaxF!$v~X>1UfD_Z6#BY!x|ub#F3&1%(+6ys;ciqX z+j{CGVd?YltKt4ZrzXt8L<(r8jVEC_Eq%IlrFgQu^ZG?S7PW5~T2>I!oyA*{HE@Js zQ3k1?s_I)X*>JB44=jIxxiWbHcu*4oKj;BpT!&v=g^hf4S1@L$S{t7Su3uTW zoZQ!4Ni&WD8GE_!JGq~+&;>Nth843p?s-H88^VK6k=IWR^J@#;$>Y3rE$bz(wu#y` ztCv|$BtTyAWKuBAU|8Z*uV9YSu*8!|bhW{-c=CI}H)6ma*EncS z64CoMP-(u(gM!P@llEFaS3x_DL0Mq@C$n=-s_y7Ki{;5@?XLdfL2!P6brdO4W z6Z6XR&PP6_)qMcYrM$mQ7d^UJ+;DUqe$!XFZS6PlwJ@%SjDdT$VKXUUZnUk`}1ABRvfm*S z9ECE_NRfanOcucGxNu8SzA3{hLlZj|*A^}>AJ(gxkPq{foayH=rgo zH*rzXpko(MMMXXNo)2y!rCVjw!Wb>XT_WH4E zH$ON7yHyOZKA^TJc*2>^lFj_`p}?jiZ?O98!A>z*0cWj&qeGW!hKvg|OfTi1qdGrT9FVPBSZGGXeF zJI18j-{;lj5`JE-8`8g^tNRIX?qTdBMiWGb^_$R+8I^VXw5n;y9`J@OJzze`~%pL}d?ZU;o4>KfS6WR$mFY(x4a1VeXETG+XF$UJy}Y%lLOIw^Vk7lbHZ23FO<-clL}9E$ zI$KAAZ?Zo&HMPwI31%7%E|fo5NVwTHm%2t$^?&Re`E zAS#fJ_F!Ox#8J?FwYVm+@6*=PPa#as{$1Y6a{2E`W&O}SnVn(>-SeRyeH-+0=lhI4 zO%4EkpHaa#Ygz{7f6eG#0*BoV4aE}yLge>D`-IY2m_cX0L?Zm`CX(zhp5cyA978$xeYL%hmX}{X6-Do0j-QE zrI(vCI}qxh%cmP)5yR4;$_GaWL~3$6#PVC)03Cw`3@G*+iOV0@8m~+&YL7hMpz4fK zl$5gT;sao%yJkUt#p3(o%GZqAv{2J+A}-teQ6g%t=*W{m%cK-$B%4Bkbt_b3R%nQSz4HUNS|Gdvk!KpBRG?x=l{;e zmZ(~$k!V-Vbgx^;kLuPiu*{fp<($eK{_C=0``Rd+wTS^z}@YyLGxG_^`fnFsx{xd8{(a z@5@X3Iq{ZQw=6QG$1s<$uwzxBDR*>zR=!qJ;0Xc;}I6vsOxE zk9{C_%*1x4%xUXvt;c_0ahB8s5q&sbvO`ZsCyuA0L_FR-moUF)jO=%B1H|N*`K7mP z5eaT~{W(Yuiw_;?766UcqRCj5m67q@ODgWTG>2jSlZpwl%Ien^+NE|rp!-qoUbw-k zXp?GhWgauB)Qqt52bZISq$gFbE$j?_M3_ZCA>LO0n4yVTPp$~vVyT$^Yj3Nr0>RB})Eu5;2|&;G1cgsHd{l} z<+b_c(l9?qMN3#l7O89IW+2(ZKpy*ai$=_7^+x=ODw%FCO|FcdI1`{n(bZy*@r^6I z$%rd07h+AsPjw;itxWK)A`7&S)6zme@3vLGCV^)9UM^;=AZR;0QsMF2b7>>7OxYYL z$J+@SuoD(xSm8!CU@1<`snI5q=)lfzA}aYbz|18N2GnPLf5P5H6SOIYNf}CbFi*aI zS7#l^tw%9G0_~h0W_-Wj>@&5QNvbqQJv^g+?nA-lMR_ zy)BwVvd3S@p43$>2tuEOy{q-+c#|x%1}k=)qeRNJGKa#lhkGj@S2pkhdpPx9d-y1& z!_LkX9|`v+!4(M*etJTnAZu2y*M+Vgn`Ftk*?tMN%W4b)ha;R7te$y=utj+=TJMPu z;l0mnhHDbo;-UQrr7-8LJ;3lCs{MTn6N^NTl?tAmzSHCBQ)(n!ZNU5*nz9(6l8 z4TuD7IJl&AC#Gh1KOIw51wO&^*C%FZ2v?*6{!6KVCmm&Jr$|chudUL%^Ie{Qzsl=Q zA?>F=<)%_}ch3h{{qfD1$_kXEauV?>)Juih>!#y1@%+&AZPlc}^bO#EPo`uNFA;7> z$!cw*DJdJMJ4N;B07>n4u}4|V)CB2a7Ghz)2JMrT0AR06kZ!->#Pkc2R!)NRC%ZVg<)wcs;QwRh-ob!G)gJBmtHJ%#rp{PU;xpRJ%pCS#O!Hqp`4%67 z!z6g_{>#ArhPQ!%1QY-T9HIPjimg9=@?;g!@66F36aMq72wlMZtYYI{r~Lfae;Fg= zWF=sKRk*ho|73opFHZ#Y74&NjkpJnEqf|imG!eQhe<~lz>0JU?L%K#UJ&$C)c1|+L zBV`;80!~~fVRE@FL2#Xu(bIE2=p_-wO~#iu(q0q1B=Ul1uU}~j%X!hx@laB_B+Z~C z8I^YZ2I|Z!CH5G(sADO(H-c$M-A%MMF?ncWksUddySks;)G66hw;KGbi}~kNXr8-x zH8dfN?vIATh1Z5TWw!ot{qF+nFJvp>_QzE%J_D4|+GDS${%Nr;R>)Q4?1VV9bSKxg z4q1~%H-{#+hS5n{tc*^~G$GhK4m_N%OI=R<(aq;?fArg>BS~~}jf)*MEo$%9Z*gQN zab7WV@W`b*clPW|3cq`+A_EF+Ybtb3L)IrEseebv@U26DiMtPOL*7|!`;Xpq_1s4k z*O)YPnPWcHy(H48NFcrWf0xr|gTG7jeJi)xXxg~Wnn_dj+P8z8x=#@GEyTj+&n-!z6rJh;Ov(81f_ z{j|VvKO5Nq?=*mX$|$dq|! zbmNW7CohIFaI%E!?2xrtTwgkXx|LZ=yzk!=aQSjZJgwuF247{uO%oxoLTJqpp|xwE zVzUef5{X=&ODtAi9rwx5NWN)!IIrR{8PL~XV5my%uy-Gr?xKTljaBQb*6gkQtg2le z-=6)+CdV(n@Y1h=Ky+u9b5oTD^K{iuIB$*}M=|&v_=?hz>eWlT)(p)J>Vu-O`-+#OqbUX zE2%wCC(=r&wDT$bIj#F>kW)KHizK(|gvfq@t5MSS7ms-Fqg{5V4(@cIQN*bn(Fv!D z-9CTot^}`kqQ+q3+=S#La6%xOIEf{iO&Uf8x#%`lQ{i;9dule)ku2V_0*Z~xQcoN< zjsKGy11OExFNFARY#{`^ws{2>!o1WS22QI~6oAwovDieit6jBjrW#yv79K1#26c;j z#n;Kqo&tnVD8uc&YsK-9>1K7coti-U;V$o#-9=uj8mB=O!9o8yQqcodTWk@tbt-y- zP-)`PhHe?pr{eA9;99Kdj7w_b=kgnask(l2`Q|Mewae?nF$nd1j5WKorBU!gI|7_? zGZ=9*3Jdm=7gEZ{yH2hsm(Li4UPhzgXZs^$}z68BlTG5?3Az?8}I&>jx}>G+ih#(uwO&Q>Wa?z%F92q zGQIV=`<+OFwYqNJ-6krr3xdD7u7A9ZFbJ^iOVK#>YS^bJtMu{fVat`0md(pBZqG4S z1QLUA7c+mJG7R@W&)^$`>#y_(jw>A?@$BB)vc`>O<|pw_+L50atM{q(-ao|7(T;3v z%+_u6#2oIdUO(L1LKG1nsBe#!#hn+oetXk^4^+RmaIVa>4LlrM90sp(!Cgj*Z)A_@ z@tV~+RP}iOys!VOH}gAtm+q==SSi8lI4E%W?)apx&K{;fT28Q^Deb$_^f`Jr^4yB~ z6sx05M2cX5CY`7bI$YM*I%{vSM3G&kBd&0^tS7qzRyB4%t^s)bb&AdIT#ziNB?+OT zq^epk&@1%3-vFHWsnYm7G^xm4Fxm0;YV^SYU4M=yIW_;37INM53(uod#P?7C>+pUG zx1@OQy?SROmYFI7%S-YbKVxmzbG~{bj33Yhq)<>$DZ#7@K ze-6~IU)*^wa1QWb{(>5DhGo9|<}WaLvGiwN@)*`B*#qaXj}=&TejE43oLJ}t`;in} zliZfsoos|?eJMW+>nn{Ff{))8vHQfxTldW_t=4o|!@cR{tQ7-{by!T&SmyYEJUjEt zsZt-WdXncN5G^S;oId!lSm$2(n>RNcWF$ARZ(eUtgVFsXv7h|I|6#GeMIRA&33L}Q z(4^iQV04yxGA6yka=gyt^(}N1E{HuP7E`z#1(Mm7TcLG@whhiX=j#<(9`3nz_w1!D zmX$eYJVz(EtVPM!7Q#QhdF8yK25ibx#*8cGAwWe*%)BZD6B~}2^9kh}pP5VkWJ~`D zxA7tI4Shz4Q3!qC@(uB{#aruh0~`W=<4I0}-KYOFVg7P&36#mNl8QFk0jSd-K7E(- zo^U_iC;C6x`^B#b_*Bf4fBlcK;w5~n7#~9SuebG+3wQ^`$BJ%Et-OC6esd-V{=Dgmq z*M9D<(Zu=FdYSg9X8HGepDc}(3(Zh9_!#Q!xpPYovwrzFV5~8hbBu)5Z84+TZcCpu z?HywFtT<$-e%p$lUVM>JPd#-}%bxrmj6>y~5CucWZ;8n$!QSmc3{wSD)prm%PkxBl*Wu!e|Sm(3(IXA+ixEEB1$e@0jOohMw)q5JsD^#FptCL z8~atwt{tWA_U_$(Bk?L_WKI?9#ZQ!vT`_b9X^*pomZ1z-*8f^oxqAV&w-BZON#w)N zB#Bx)P5M+DPF^s&#SUu@S)I)qDGKu-S*c%qpgn|I4qKe9U2PC)!7}0w&6|~Pz1I-0 z28YL|MxXlimUC}7OuJTz^QKis>mTjEKG+{(Cm_7jP0!Op>yvb_iQaLVE-WeDA=1!! zm<66d&z5WV2&o=#v~R3;I)hd!o0hA_%-8eJ^@#7d>^d(mjA7XKi3_n*l$32jKH(QE zw!WpXw^iSH-cdU12k0uTiQ)wRYQ^Y}AhO|Kn9Wxl2ka?;;)+;_I~eD+o{LW(@rNJS z$myJlF|DD!fiMGb_r-hOzdXAq0v?^A_0j1WyO7V#1$4%&9TAZE&qF5Xbau(sEIMhI z`9FfHrxaovA!S`PSIoTiQv7h#f_{6DCl+7dInM$q$ZdLh_u^6ap7c_uStkycaqqG0 z#`;qGF(Yc(2L+wUJGDB633CMu>`+WWoaAcF(30$z5a-#nCl}(ZQ+%|14{)^+^E}Wd zenmQV>F{N4T!n4hJlLBPUbh7vN~&sG>Mxl-5&*uEQG%xIOA>ErTZGqLZ|Fi&ij6ig z7e(+SUx=yWPE5YPdOWiVgo&&5J8^XzE5=sWyq0QQ7u%9N8rpzf?A*J%y8(yDUsH1} z^)&Oq!eQfU2jd_LF$Dqq?wM|O6R-gSC)EisUWw8*3At>(__R z-P$n5+$KG-BVOI$;=C}DxFOOJ$eL%6f9VgXn0*xk3 z_1p~p3N7SxfL$;Bp*j2a$Ge>>M=LY$Q;AiUc)*>+qm6}gTKRgOj0SJr_b#9$07L^^4IMa@#m-cy2uWV; zD*<&3&6D!IZ_H+|WAR$jyyLbq3L$uAG~fTqKql#^$i;QRh9<&pqUcF(BgA1ND!3>H zpa!%ez<%^G0&vxojmvg5=jmO{CW%JUYZ#!FW0kP{0hh;tBC)!Gi~ZI4xn8!1+8c%Q zm@78nUoc}Fs%y-~g&VFb-f8D$vQx?JI%6Ekq%9X}x4&I6ee|r{wi+R|{qfF}OiIxN zkcw$%+d;eg(#-CVr!9?CH4Jp+E!KG(yG>&08)i0-ycnYAx{O|6tOxHcrYq9>hM4$m zPzo#^sLo`N@10)wAmR7z!I)*Qcy@fF)b2MHa=t+_&rk_>iFY^QGo4U>kIfun^|XPT zF21Bk<^YNo!XeGer=AcD?WYrwr&d( zY@zPiXvO0DQUE{k*s4Gzy|V-D_Tl#QxfQ+dSW&5n|*5UK@@CNs?M82P!)0`yqW_4KLI5ktq!9e8e;;rdkGYol_)0qF| zT$xe$-WU$4cq+KDyFp1}bz+d@qJ1>j@6d2BY4{HPwd1s*t<9q+qk$3g;71C3$-Z@+ zE|utk=}%_0h3bv1X zx?UnR2YKf#-@S|Oy2nq`+S#7R&dgiiPbiO==ZOxm-Xe{W2X_uiWUF5jDL{;@yfsq@ ztv93+FBA~oa&1-|j4Sgjc>I_EZHuvISkc)?yQMh>UMxd|&@OzMOQcHk3~|%&cb{$! zeZ4gpy}!k`VQ6SLN3-c_gtX<-8av8^@A4-Kgx86xV4pwhcCb8 za6_qu~;fZdDPdH!g{#cQ)1O^P0w6C+RY{21~q?>%}gT!w4-Fp7;dVL0x! zhlcyz$`Q8Z6@4sZxCbJfcT0zK-B$m7f}NuJzQowZ!qxm=YZ>q01T6U&&R%gFQ#6YRAdjhPOj?pgy0P8@bUHD!|6=3u_+0IWQnGc zc+zAr*n1a=bG12$Hn&_lMV@gdFw9LiH}~{|`L0x`CquudrD(QpMwWXPiX&TmYv-S5eFPI5O0~_KC)G4cyX>6Kp$S# z$JZ%#C9JP+z}t(dQ7D0UpB5f{u`^MPavBhWD-ff8~B5LZFj z7HS2BT4GGO}D~-M;I(>vcX%XcIu67VZgk+DpkDY!R_zK5hE4BtgL!i zlcCt@zw#@7s&76ka5kkJ)9a9?Pgyj=wyl)R#pKKVA_hhc_Q{4O#b%DP;-5sA5aPF# z)o~R91c?@yBG}xq5i8cj59xi_bwz8E(0pge8k~JCD}Sb~g*_4GDFLSocdL0MvHfK_ zGmpG?Y>My}EKfK8ZEa>(VsOj#ijL)S%V3O^-jnn~qY4pO0Gf8M;cG{{{xIb|1VjEB zO!;1VbleDW!p$}k$bu!O1BNs16nwI`HZHe;EGQdl))plIjVK_3|pL9aE^d$0=DxD=Wl4<(m7QRz;6{jg(8kW(XNw?KIWl>OS0`vH~n^ zZP6)yM54>{s>N5?oaeh7YuLu`9J%#Qs2^^>>?jh=hu43;P`&W-TGv-0nx|vHSwdr7 zL-kom@S$}H{C*QyFhN$mCb@9c7Jko@4r-ZkTtWFC3i3ndH;cTcc3~}=f29ISFZ}D3 z^QP-<76vym-iZ^ZdZf?}q5@qO-7M}SPm{wImO`dqK68ZsL7O8*ru27O@YFe=@`@D_WsQ>6RwVIvL*9oCFx8*-s&|>{%o$FEY+t#>R-Z0 zMqR>kvz%#DTN>wy8`_`+U|)ACLnkllV$E*e>87zYqORY=DM4-@S-ofeBQ6#UfDagp&G!TtRPq-j^a6@nqy!~vV zI0@nWgmr6od1okV=T*{DR+g{CVf6oD?=9n^>bm!F7(zk>MLeoW1f-=K zr5lDY=kZ`-ww~>!yFV+uHb-Jp^s&E#B5z=Gc`act= zuUmC~S*h)Plv{6U6&ZVUi)bH%wglcxy>K>=T)(pX=Fmze?*azV<484D1$}lpdC^P2 z&?Kid96GH`9IkUo6$imVGAVfImDFtGJ#$>Xj1;pIM@^p|`s1np1kM;(I;@T3f4x z68FrJmRLACzU_QdCEfRP=@ShDJM>PM)#T^3*|4^ZBlP*GirBIf=)&14$&uc?bg|t@ zfLx{&QRV(|@Y9uwXj}Qe-EnX`iZf95+H#S%s;x|Einjjec4&XNxw?FT6R=53L>5DD zamwu*_e+`AE}iS1pK=gJ5|C@Wul=EU&>F{H63;OLuYU@ZfiF<2FNiY>;QD^A1d7hU zt!jL-z5F`e_Rs`^tl^n{Ub9`3eZJ3CUQud3IE24ZQi*CmpWhlzr)s+=q+mHkRd6nB zeTXAj;zqOHTt3Ts?9eRD8>N_p7x;~z>Z|xrU$L^v#HXiL7U9@-thsbO~Pe0Krk(0PV;xt`$YLgd+@XDZf*@ldr0KvcIP9cQ5bLvVH2h^%_QG zTl)(d38dX1PfQlR-tMylMm|}#J3H4w@H7qNVP=4mVpWR~)p5V{6lWewf0}qCLVCKq zv}&<#QOzPE4KILu`?W12`m3c`uy_04(3H65~de`wEqu!bvbX*k{47W6-9DeZHzR6z7>IDwo;5|2tFd=>E z8CEsrl}Q{WOQ~L-90)yD0kF3}LfDNv0nJ{1e!Lgn-WCsw*52epf-gu*4D5Xmi-{! z;e7hCIt@h^Dr-=|n9o2w2S;O1ZYS-_8r|f!jfh{T^U0oqZBhx19SZafl zV8SYMAu;|T5TCW&N?+(ml(OLGh-*{H{C02U?h1!m$9$e{G_b(dojSz#mWLt@5?n{K zZ)^m)W|AmP&g<6_y8)#913Koy%HEeGx#`BXp&7-^**6vzHWDg+I3+yhJ(s(0uU_F= zTf=;0W0ikq5Qk;Y_Y%+0g+0$PCRc;VnMVd~jD(Iq3B2GC!Geg3VigY^S1-8opk2_|UibVR^*#!djDq?lpq!pz^1!z&vKRe* zaf+!Ov*qIj(nG5u41YJYA4JHS&&GYztb1yG$((56MzQq8J#-1?tB4n?7GztS4*i&{tS!?U)4tk|M^DJW=U{1ls<0B*_^jB4djT@%Z~D*KNzMhCBwS#bjOOe~8)s!SVsW z*1>qvSaDIl{~$*HLHUyci4narPie1}BKpS_zkZ#92}ruq(@6ibRsUps0fF{o{QpVj zx?2FG=JC9o9M)eW_AgIIAqN5%VK@KD9{$e>)wsaE%3-J4#}i!}$>2i8Ou#Rh}nTgWhFD zzK}MrGYXs4Wedk3WNZJ_vL2e~de)n}(YFMyT>MP)h^eHUR`u-N1kbY2%X(LJElvIZ z9Qc1k=O8|4O}SqezhL@Bq6SKbkW~?pn;jTesMa-4oEDavmqj|Ce;JmeWRf?p-y_ae z%!?Vwk`*|!SuxC3=M(_y9uXS#SLOLtt^`B(=Meta>?D%zlKcr5S^*ZC8asiQ(y3xxc%6_*Dqww5-dJHups%lQwu+EG67a)Y=U+vNZfH=7 z4(1riDJvI_zxiiKDj(>_4^Cx^xrHRsmv?LryD|e^f4AsNB7+8+cvE9MNb_ux;~|RJXfRQ zILkU4C{WcL%2g$+^3MQpZ>&+9CVq6Z9H*da=uplJap2e+xe%{zS+r?dV;_86+(YARg!DjX?w2P2UD45fB z>!bTn=8;s>{<1cKsx&u2MpUx~>Avu+laJ9VlD6CPy3Ff;_{^z2fspFp5UV?0o@c2R zsIfg4nFs`G=%28k0Ecv>1G0>p#P<1WtlJz8+u3jWc1AQpD)dSKOoATcl-HDQEJ)8u zscIRyfNb;X)hx(f$Xs!fI4Xkj?x-Ud*$%aZ30DWUk12OTu!DCAOUBG&?|O`c8CCBR z@`Rbkg5?+)!45?>rii=bKNRw$TZ9UCWuMTQ$@P*)DJm51o<&8lW-s26prKZFbdwN| zRj@|Qz`>!d<;_T!q^?5j_d$!8B6+0A(Yx?xUw)C%%yX)O@vHKUSE;$*{uZe`_dE6ckA3>4e^CJS$X4 z;5pkB=uHGvW8&uK`FC;GZ<;c#7u}dG+R`ECk)D5AIcIB*w zOk^>EJen2z+zWL3U$p_BZ@+g?s+x%l`=9ml;2_U|#97HS)`d$J?O!(aUr&OF0Z2uK zYBlbX8UKf=y8i1ODgXrpkHw*1|KqHgE0odocu@(LQx-jc5 z00Y85e-r(mQRiS_i^b0Fs{FThUH@B>9I(49qH$9GFAD*5q=NWt+cUghV`{IO)Zu=s z*kYqrcW=UGdvZEFC+_J>x>aeq6fS8fVOAJ_Q^BSkbO2@K zfd6fd6@or2@3SaRKUD~NstoQsy)Uto4=+3}1Lj>RDS92kLxaxN759`aR}$Jik(&u+ zoNyjplQ$F7Vjj$YHg`EFq-gN9jeW6vg!fNvmVe4wB#g#N);3NQNOM}@+BFPJj#OGq zq_m<{{~KtxS0GiUky!K3)y)xs>nEmBc7LJ})W=g%e6p;tBl8sN7#O%uY@$96W<)s< zbr)N_)o67&-KQU(@@X-DphtdriRDP~kycd=`dgfJRNVC!bj8=VC*)q1!Ao3rCiRaz<6@IuryBZQf$EK~w9G+#LZw{$2g}~{z8ORpN4g7ErFGmvMrP8p@r2d4W+74u<>N zc3x;5fU_MQ0xOb&KicYYIHw;#+EIKWzUm;P+vbdqX=ShZC`DGPMlKtSb6XU;r$RL= zXlO>a7~030^U+y}->NSpj5$|Ejvl!=)gTmwqdOAIeVQ3wERRr{NT+{@8=lnyRN5)^nanv@(rwzleD zgM1Gv%3B?2R8cnbh-ia(A`SY*D|Eu4JWplKHKssBwRz0tqA{+x_J@$Iz6={q`6fU+ zi{zt{Dui(~oR{pQkd_T}C)MEVMVdNxxQkt5L4X=tOMgk(Qp`X2YgN z?;AlI)b)iv6o~4C}){S3%9legy0q z;rXn3zt*&uJF*}Pxz6U3=Mf!hY8nkio$=T;#kp@-OoN7{SWL)0)2lk6{)J|h%ag(t z4`9Yo1J`~r*vV6$H`7R-5>Ca?Bc-a*FR_jZ(hERF;f!BIJtXDV3bTOaJ_C27qSu-t9neZi4AJzcAusrVVLz6wZh?@|SoNJ;**F|f zg1b*40x%FbT6_KGU6Zdl;rcw>WZZ>**JB1hNwpA?RB*>Z3JYd5GVxWOsmG%h-HxR- zj}WP;!CIv$9;+ADIA{Pg=bsle5G@U!|4DFH5>CA@{xh=JngWxuVqsXmedzj~kOzoZ z@A;Fo3?OMQ@(4z(V35=KZ2xD&)9a4JW(+usH0#!vRMC~)Mh%n-=n^tvev1k6EFQ!j zaD4qEX z_ZW7+%+p?m7G}q487FN5F^c24C?N|xV`&M2w&(R+2^lZVm>zvJ%V<-pWnBsxEH2O0 ztPvfeZ)NI;aIH~sb+CMu-D$9(8>ymLqrO$%I21XSdKgirB~cjDTv_b8#|H|d}zgi2BOe;l{TVa=>A8-1HVzy?D%C=ab$6uXr~Fe zQ`2+qtq_PYV8=*bBJ!NV^b7w7eih8CajFE7l*-(u?Nm=7Y#cHr2!I1H>Lo6{#H;`I z*+x)OpbS)|K4VPm>g@30)5^-sY)8kq!oWa>UZBz>Bb{bMgW-{oDFn6?Pa6?;o&Bx_ z01toAm5vPF%P5{5K~uR2r4KiM;;GKzkici?!-uDdQuX2-R-rt)?cPe;jQ}#dpHs1 z01;d~yPkPFZ(@$$%4hc5G7v9BA()$Ml3yiS96Q_U6JBX~Ld4cOwrA3XvskK$$F9>b z3e=Vt2a-!Lw}H}|#rkW;`M>J#&!HI(C&Vy|q_zc=JaO1m*m>zTx^8U~k8HWW(*Nxz zBH_CSg`v_0yBTT-YJN;J{pk1KX^$plx*Dw3GjaPX&hJizP>dH31iUmpV5GV5^_rBm z7QGR~iQY`;V7BV~Xc4ou1^nsq(gDZoa!HxnknOh-MctXZ_siEMuu30(U-bn$Bi*@k z&t$I+I|n=m zx6GGWd_qRg9cO#RqHK(i?&$1-porZl_oV?~Jm05+U(HB)q`cXk>-o)ZfOF7B@Yc(I za+Lb5d3Ov2+f;I*#E5mrl&(HPta%K0orWtzYmd0O1=efb!km9Dz3V%eJ$pJEHzQYO z9J5-lNjaLTFL^W^&(_+_WFS(uJUIK_T-d2MnCrsEH}N$f^RIPfV0bC+QpkI9tq-NT zZq#`t8Q!mpqq1);R(WYM-B0v$U?VOg1c=|`H_E*r{Rh;1^wHhrNNFdtQO2&XdrT)} zDv`Xhqmv8div8Y6ivn0Z2%NO^NlA zb&~1Gr(TQZu#4hkq!RE>Ka3EP_x-EwCrrn{0mz!C6ScH_y-(_$O>yPn2_KcobK7TW zQK@e>hb4Iaqdi_!T8RRqUE(}}eRIdA?L_wnyN^F}1IIAc&;Y%)STW^d=8;JDR&%-Z zX00qe{u}RJF=^?@ni{(8$&(F{ZsOWmcY5ilxnZbs&17}j?1iwMR1Mwb6X`m#JM;cq8$&Lq7XK+1k$v{6(WT*z{Mc#YvEk7?)Tnsa>N z;X`5_*fBX{2B-3d$0wrQ{kob%_i8XI$xx7dls5rH@Q4oo@!(D`k_FcwJL_M_YFZlp zz>1RYRT_Mc4E3?!<4gbi?U)z9wE~z5peP2lzcc;N>zmC?GqPneaPEE8SGMv1NXWm> z?zH(tz-F9?P2|-pNLu=fNE(KEsTdX$dbgiNrFV6N^fLy&i7=FL;ovgoYH}$mQHQbI z)@x$gZ;xso8jw!N3!qt-tCc*UKsIXJ#nC_RE#qB%o52>fB{t)TdrGz45sEIj!v*tc zYUpFixim(7#^11!SAp-K2?sw)qHizmL+;l1cW3*23fy! z68E@~Fb(Dx3^PnVGQpc|J9Z$9Pwa9tQ{^g}a)Zvl6JA&^55Vk2-8}XCmX}+5=T()F zRf4I0R7f#B!pa|1%SDY<0GB1q-U{^E^T$C!4O#Qq;tL4r-5*ZVLYYpNKXyOj<<;al zTvTU2YLeFlpYv@us{;9c8kBAOYe#&ZDULVuMlvnu64>N+Rk5nQ3|48^f}fP#DT=13 z&NK_o%Hx6N(b!Z{nB`@OR^#_F14fc5FIO|zex4_u#J0x~9@=r&>80jlL`x$$c(D8s%*j~!aQ*%qFUO5U) z>c~|Q>q+Z+6f)nd+gGCx7DM%NEZ4nl3hj0cc!4hC#03PxgJM9asD2rl!y)75b>iiK zz^y;Co(7Z4Ij~pb)9ZZa{i7C3m3pSRPIv?H$%wA)i5=<8`S7Fxz=*lOLtGs}i_yd2 zUo_yfKuF%Y)ftW5ycN0jUEbnznRg6cb8sOJTETeBR2tI){SjRGub4XY>$Gg?-qosT2!d2BM2f2#@u{le)F5pO7OEw1eAcExBmf)i4$UdE_C>jXSVDo3n=$`&?n zHBulw4WHo**S=%~+CF|ZJ~**h|4OoL*J4EX{ogW8cWJ~4c$x8IdEbogK+{e61!$w} zX|4?i%ss-gfTW@NHCHFy2WmS30LB*NC~sn+1#Nsc690fbFs)JZv=^*fY+WEO!^~b! zGn=!GxQhf?jWc}{IAwSa?9!%4>IiA;LusyVm1JWyDd&N`il2_poi*y-D+ZtbSp!&` zneKU-Moaui;0{NsLEFq&vF?bUN0LbjDE8t&ETbYHuTqed3d!CuCM=TNJi2cs&*C6` zTh&>`mZ(AofhK&kmY%QolS>zSVIYzivS?WrqNu0l>yse<;pNx~KwYbF=H~XAJmQ)v zYT6CrgA5ix-P+c>h)jXG04;yBe54Lp5D3J@ED7;lrG5zcq<)X}Q%X1jM$V%2LyN^q z-0FNJE0pInTB;w$vQH0I%|&xs%2!gk1BdrZFu5n7Dn|IFbB|&9Jd)S*dBBIDa{45v z=lvk&pG-yMK+noJ0JIQIcLEdHG&_7Pd*48o&MA1(!H^&cFc%tbE{}O!I@#(NOdcVX z0{M^%x8erI51TmOr!4e^wSr9_QJDI=Dho&L}QqgS1Mmiqf_82+*(nA=z%1GP_8I7Y~FtdVt@!J_kyIEnkp9^FRubSy~tB^IGbuNwk^NIAsoF{Y9+n`-jR zloAyIE-980{E3CCwDx(z@;M$kYzA$lrO>jJID;osTzT>^B^b@~rr#5o;Y>tmZ>mV7ihNQ0r}bgP%b)Mm)ye zm9j2ocbFCSnd_)pR$08Yj(2G=s_@lcb~UUr9wOKY3Y#`$4r!~zZ2Z((;EPPZC%m>x zi5rfYFEBd{;l_IZ&i%C7D3kTs;;oQnhqYt-*+v(M0|Bg)imB5SdD^q{A3TXp8Xblm zMnt=fz6rbu3pNXo>jNz(*x<5Jpm)pmyx`#b`gE4-{M@p_(OVfQL@X2~fEN8EXZj8d zLSlx~w$QfD;S;j!BP5ZuC9G}h)%8r^D8PKgGI}FoUI64e`Yufc*&`JdE^vox)h;l0 zaWQEx9iEW3XGozp02Rm<;&9vNC%p~Y-$+R4ICC1V?>?@0c}~BAE>Lu=y6TFhU0qbM z(7U6g|Kw?uC#Uj#8U#AZKPOtk!SkYj+BhkTYXl`#aSfc-yX4Q+(0yK=*N{HaUEOVuzsnB#BJvKIocTR{j}jTxwdM5WWl|8&M_^G?FiZ;_L4@> z-u-9Uc1_TgJ9i~l_r9B=C_HZfTBZX4t1~wRO~0O5*VoDC(mnMX1S7OZ=PwUDu}-j_ zAK2aC^Yp)wGyX^bZ5|xNjGuqg;SUQgLIHo^Qa|4@c~ZS`VpCa7oSX;fbJX9oub{$Xc#K7m#=NS1u&wRO?U9;rScTM;HW{~KCqq&bs!f)3{?<&X zpJ0`?Ks$99jDT7*gHPxd-~s5r#gK38{UjhAbQ@kzV-Og{BVF~xQn%22PrY%Dv#vw_ zqsFG^Jkv80uS8+*On{8M<6UC)_GTM(ABP&=TqW|Wu-#v1nu^Mna)a!D2y zZ@f$0)8@T0 zagcVro~ml#l(4?+wLL-J@dQokF{FHA-wVYVmK)G3TWs?;QJz0af`2Z#4|(;RlX9yW zQ8@JQ`={cJDPFU(*{}Jr>R&fIbIVicM;$wnZ+t>dsPxml3we*U3$(*NWSM~v5k2H7 zu~CyI)wLAL_o41)NXDKi`^?8L6B7-;&+&Zx3=3s!z6Ij^7HX81(GNk6CFyh`rR32h zp`6d!C3UvdUt#zxdpcHeNv)Ql?%#RdhNh%#)dU=#1+1r^_2s2c`TF`OX#a%sMf}_` z+?PjJ#r-%eR3Xa)>Po}iH9fif7mLLSSS+=6U9CldNTzsIJoG7*Mgwqh<;Yv|c8wbK z`iO!w2WZGEiY1o76JSYB?Kp*eaxUO>(T(6;VQdB^eB^5>`_HMcs8MoktR%yz4kFPjxwr z8!543r2!*#h+X6^c%Uy%QJqw@Ba1?z(MuY~FFn)t$#bWLwf{s>0g}xF30&sai_CXA zFW_(iPHX`t3#!+69~4fJr2ZyNTT6_@-1v#IYCs=`=$!NEZgr-5(fU=D-15HfS%DCHjv)|T8=rt=Kg)iHqx%Xlza|P zgVhaT`O!va)H^@LRR^%{N&<*%(W)2PYO3`Q7N7gOchj?%Sn|nn*E?@@4djv)T&9d&$>7QD(hYk0!Sxp zEj+cj0CnL?V&sAPsk*U@PqsSVbtoZEUn;r--5l5ZmpTvz%;Lfj9%3cHHD*ss9czV7 z8@`TJrXAdf4_InfjVS7wy?4>-HtHk>)uq5X)AR!zE;aEX8o|0>7_*@O7MY0XEeF|D zzzsUsy;k7OT$Yh{JtdZbh5CS~3c&RLc=14*>OoT8z0ztn72k-u2}EJ>*d}AtscL0W z%lj9Gg#l&JljTG%Xwq&gsS91n|My`P?vaR}QQeIl_-nH=7>bNuhdHq@sM~DA@V&h% z48RtUR32oxPcj>SpHW*x&uWw%Z%@@duuneFmV60P8SSArI-CqoZ{oOkN-<5 zOLs9Ua^ek8M}>`EkFqooy4w~JbE|?k(V1&l1Y&b=wyMvrINXSXtj+1w}E9+qeMu1#W z3U6;D;7Fn1V)M1cr@D*L^H$Uvi0yF%pfPGbY`Qxs^^?bN^4N`an>Y6!FhE(~8GKc| zSaR7^)8^b_!&Q?%Z*|6=R|9--n^7i^JDL}pE}n^?_odl?lJ;h$&_&>m^d`I=+(7x+ z0Ca*5Y%>XC)Apmq?qzEIO4x9wgHe!DwgK>JHP#=Z>kb-KEhmG}XE(nDgn6Sci0kWT zbVhB(`;i>vc1E+QQgE?I0r_tWCe~*~KKw$}&M>lx;QkzPeQ6#5R`X=55NXZ6+_@Y) ze$>TN?*N1X%-crX1Y%T25hj>|+yUGgu7_Kyuqq+*Dl_O9 zOZGfr8>GnW_+e@RI5jdRbkep3L*Fo>v~<~WZf;n#M89h;kT)WtUFL1Sj_jJ>u3U%w zE50wsU-v>39-E}Jzc*S?WC(`!+ zAQVRnjXYuT7v>HleNd`zL?_#n7cz#&4WaC&<}(tp9AZYZ`bV`RmS zXc2kt#EcYPLD#4!!ehIw5!~I9k%IjZcmnzE}gUL zX|P(roKeK`6}#I9E%&Fjr>Uj}z;Eq-6PnjBKdtG);WpJef9@knNxum`Hq) z^)V#`K2w{RxHuS3O?Jg^aWV1l_uZ93?;+9DczFDo%_tIXs$U(qDCP^ZVn}s>=MufI za@Q*Ynl|;8M(aqytV$sGvNx;FLGsu|YLzlR4k#$&LIGrT7X#X_wvKVyFYnYJ+WFVL z5V~2f25@!sOm6YFnd=p_vzw4lwVzi$xqa^0LpV=##HqKRX_eZUUSTWlWh5G`Mv0qK zH#VpKk0L<2ZWdvL1r|)0xQ}B`QL?!ZgET&4sT}{OOxW(IOZ8aAN+h1;b8T*NLp*rcw`401Khx*| z@hiQZdC8|+qb&$4QG5*@pyRhm<&*tIIU-wlJ?0ywAt{&)@whQ5U-GOarHXtHzk0~py6Ro3p9solm7U(0mNvxYj z;;D(K?kX6+(u3@UNgqIZf#`uHe`2afPR)5Eb1z9V4o3SIeoXYmFD06pxOSLpGurml zE7=712eqE?$x6GtXuHUgaBo2@{m6ORTVIJlxfijdQF{ix82b;y zcuWAFu@0r1i6qe(|M5k>B)A>*k)#&$+$>dL0TN8RVchh*(0euvOwf4Vnw`dlzsHG@=UJ=#{Q*Z zEeVEQ6wg>lZvZ^j4P8g8Jj^bUaLVa}=`6)nQ@Df;l{uI0XBVa9z{_Zhp0XzgQHo^Q zC)`Q1mpGCfix$PW`tx9%&>cj{{E*<^1zO&R1AgoU7`zW(3qSnaj{0qKpIfD8s_Vy} zBaEh~K8nY~em5`Bby}hO8PQMz;qJ!MJYo4mU+^y=EZ9JVNXJf`rQd4N4Nr{}9eN2d zrUd|pl&2=^ZZ<%nCPz)$hlU6Pkf6*ay?nl>J*19qd5)2NuW==Z-2}CPvU}Js?)OuW zh4+N3w<-<@YaT^@)j7Sn; z?vqU>DM*?rO<#BedEy4zLtQ*eUsYGl&!pp#sElnyzC@o9sCoJ&@^{PkYxtB*VSh6 zv8{>uhsA-)L-ec+JktP35sC*IgOTMAFjdxG=w9;SU`yDvf>E9mgpM=BWR-W@Fkr0* zFxKjm=l&c+Y=U$zHHW*$#P6uj9G}L5k1kPu)_tH4!+(5x=|a(9AK2!`XU0~=nmHDq zPf=?dQ@X-gEkUEzhWmv4(478PW(?M*mwWYI7SC>a<_&2z)~6p}d^{jku{)@6RJL1@ zwu7n;BYiB`sgT$E?H6HT!k7b@xhN`fttso-4DL#XY7G+A+x=ong)4dm={yY*8SdHh z;s_OE52F<++fKR3>@Is=8QyM1cI{4(B7@@FOU$T=_z=}K;=r!3=D13;^YM*eL! zOWNy^XB&Qi2*CY>&T`x6VZ%sy7#w5Sbh3IkB#eSh3lM*aynu0G&>4-4TRk)!ORq1F zpxT&ha!My}xp<~4_G5;_Y;O8d1&z&qb3D`bsixmaO8~;Rhq;UIs9PlP;fZ!+<=en2 z?Qm%#d~17U@~W%P5heCGGjO`J=+w-3epe#|TcR&=H_bN63$_rIiMB}h*YEuFuXreN z0gtrllH3eFJK(nfG2V#SnAJ~-#}B1*JraxPNm{Oyq$z;5lfa+ggmfCk@GBq9z2hL? zuow`ljB%!7atge>-Q0`7w%8KIh+lsB*_|FIQBdkxRYn-Ba-0Y*{8}>+byOvb#3ad4{ z@KJp9J4F)RF|+K4_k=|Zn5Oj%w#Mj7x1x|g*7SumiT%=gH;jXh*`RfZ>6n>Egd1xs zqU%6+LE3Q(`DD7@xokDtx_p$kJC)I*S1w*R2*m6!sFIZL5Jy=}l#KWI@gf#2h z3KN=Y=0JB05GAhhlV5GLxLnl$!w#6rZEi3rKYX6DVS0+LH`(Veh3p_cBty|VVcy=C zQ}@Dh0Q`m+iWizkRdplAfdeALkF+MGxBS0CDqt_5-^mblb_Q?@)f4(oC5+uLzpFb^ z7i{6qAO3xlIdnvHk1Bw*Qcjvd=A)@+p1tr6U_bs)WRE~VP>WK9@j`F)*Z1K-GiaYY zlWY&~dL+#(7Xjx#T(%C26IhrmD)m*@KZqQ4)~j@7e(Krk`T zkgHg!`oY>I9gHfEiMtJ05|!&Dj+Q z%0HUe_GY_k0N?{?Sg+p=0#V`-+zT%n_JhwUF$z_IU>efF0i2b<( z8D)fb%Y64rkcIFu2YmjN+z0^(1Vc+r63a+L(3pxszaO5PxW_~g(m!7eE9)jaXH&q* z#8^CiH~PfG&MUb<8=VYGWr^@QR+cklU5UhW@@MYPxFCf8xPeHiJM!}zQp;vioUkZy zd6sKWLb_N0W6WaB8=kAHA+3mpj4%;rW|X_4>monfS9*@Gms+~teGII_+dT%Lll{pK zF1#N1EJ;Lxyx#^3_}_1W@{I>#DY|+dTqO-JDhEsEAA0jj34R9!2P-da^|i{Mr{4o) z>a&k-}>-c1d|Kfc(FloF#yqhLm-iJyP-C)&9}Rj9&_^G(?Ca# z5?Ve~!>OY_(NWfs#7!>Sh89d33>x#!SA1_s7b9ZxOVs+OkpL?4KzfeqL(!aY$>qUj zWH=pg`$f+KZVI}XvM?ZbmoiVKJQz?PspU+tP1$EMXHAL3u#X30WzoW$(dB$Vlhsp) z|8^||cR8t@5mDYZw}hu&#Y#2qyg#{@nx|S3y1RR`^59H%dn=PfY%27mBgpEl_+=K3 zR_@ybr(j*Z!Em6IBg1I05(%22iELHR-i&wp@165<(Exf!nrUKzFnTNbeG_owRqv2t z9pI+2kFw7{_l=}y>_6bK7tc}5e)x{WHtlGl)X0i`t4V&=JCcGO(#H?0Gz7@W6vLQ% zz0LI&t$-gR$k5Tw)`>CP$t=m!Y2)O5(tqT-rO2<>z@$2jF z@7gVo73u&&q45oiOd2I*^*nca=&cXdY9njxPVjiBaRmQ4K>-tM3RYM-iebJd24>!? zyAgKp_o_z$9+Yprvs2OLmRUoaxx2)`z{4hphUQbVG3Vf|~-gRBBM!&wwGyiihLYe8kNCz1Znx~Fq=N1cL8hbDkB zp$O;wY42e`^Uz&s;15_QspQ`|FElQbpBVR}rTfUrjeCWVF$f+4RRbb?g%cQ#J?Me0 zNlw_^-EdjWMC{4}0@*ANAhKP$k>75JC1Rh5QW_i&DkZYx3_9amOYs6FPxEPiE_k!e z)nZOYv@{1|sL%t77rD8*NBn^NJ;e7=4IX~QVE}rekb7izw2XG&U;RZxOv-IScrKkE zhJhZL;qSZdc8@%(3iQZUP5r)OuQ<`;g2)9WD2i@jW({x*`+~VbucYn}dKU+`J$IBz zZ{b^f5$7{uemF=PE-7iuK;*m5D`}W0RUr*Ee#>h5zerfvRo{5q9Eg3Sr=gv2d3nF2 zIayvUsfF%c%(_j)xr7Y!*Lt+O7gCA~OWBYL!n;XGwK8-@CGa+3X@K1Ko%xob?1d7D z6sSs6)#(rCCwa9*5dFaXARsqNb=HeVI9%M-vRJ1jmM(JAV1sSn`L% z0EiZA7@Bj7=g_3UOdv33x}N$glL5fnrw+W{}AyN+5c3W8# zDU3A1CrjrMAk42+MO+)DOZav~-U z&5I9C2rCP}dpKnUI9fQ)e zCoZ*zcV&ka*+`7!m)i}@_tUp4-W;rl0kW+7Hg-zM9TVHo&rpvIqfc@(7VnG>6=%#p z{GXqXv)_FPmIu;1Nakw%~Q`{KaOq>FbW+Wh^Z@MxJkWJI7{gj<;Z6oRR9c z69_CaOu%7s z$WQiF*XgR!5HDEn-D5UN@iZ7{imp%qu!CH@)t_Ga2|EjeQuqZg+|NjU6j2mWQZPX> z5+Vo@OCe_W)?0eXv!oPv&_mJ}STaGAx9_H*+0wq(=DWRA`+>Cf(W6Ic{355_v7aKV zJJu#A=k`a&Po#J%;~jS9*PNyt?3^}SnJiGSNWn=%h=}|qYktOn6ODNBhaEH&Kif$w z&3?M6^>M-lrH?!gC+I9tDStAsJKV2}U?0rrWKg^^Ec<^Lq1kz5RP*X-jwIIzyEi65d<0JU1W{sm65&~*Q1JARl>(ZcBFihNzY3|wJ;^vV)@VyhFuVWdi0nluys`xZ$|upVtL z(eN}QW_Nt7nuUY%$c5{LqMtT997x-VF;X_m9O( z3A?(u=K~_55NB^&{$NH%wtHXFM%%b%)*yv3-e0e0EeEj#C3R13$pL2_yyAc|+vLBo zqFj3(Bl)|FWGI20PT^PSWN3cJKPk732`6ltN;1Fw^-?le$^&qQ$#KmEc8LydKOT+$ z`7cKvFhH5ydqnj!RF7ja(4V%h94^SAUrhkm_`F1<#(`U-bsZD^{ZO!$EYNy9!R{iG zSTKP|e%TK7o9-h_H&dI<=0Ep_eU}}n&%4S_Wk|ni#KV%{TPu4-J!o6 z&XDg5%~i)kmT|G}<*o0e+WI4+DSd}69RK~1M^{64@gW>f#Li;~Ga$oEXP~{>>Bi-< zgWcBb0qK)5T%UGBJjm|NNNb#Yc!TQKPS}4g0A4MNQpP^tFsAV_9o4g#A%EGG2QSI5 z?k_V3wEl&whk`{Nn7}>5plrfr{#4}pkbvF6)o25lKDeM{Ph?CV#tlECVMlwWYmdN{ z1Z!FM{=5&p1@eYp%)0>{y|pn<_xC*-oFR=XH~oTBg=b6f@I#kz{`5IV9k?1%=ms6{ zBdKTW1NTaQPTxCOk9xL~8>;g??L*}x@`~AAr;Rern}i%(36Aq0J}u3Y=sPsqI=`ed z#D*}VsGBUj_zo;s^U&?P{8zK)cc-hRnN&?!md-x7dgUHQ#Hw;(7KX~rW#tFx&OB$c zsYTy6!+%Uz(RWOhYu2B=X|Cx0P$y20gURje<~%PS+x&sjIg>kQmYd^!Lqv;37;nj3 zd}fPX5Wz=^tAT`lu(pPL{-M?vfP)qHz-tfx!3UovEzn=3V$yH_mscKDZqwCMG zmd1j=BBTa#9m{mavM~f4w#)WrN>4poZpGei-dl%O^O03dTAnDIbd~*>S1qszGEdz> zy_-S%6l=|SLBkjI`VyK1c@E>>GipvLop9`lr|&V_z@9$6_M}ws*UY|nb}ONu%K?dB ziBk_gj9XNaC33KpZ2%|OWSh*lmI;0B6^_Tk;H zs&HQHdG%)B?wjwhh70?*pX+VpEXTWRgyQ=~_L)X)2*!pB4e2+AWK3er#p++2G^ubF z+zCA2{CbbsUZy{K>~R^BefjMdqhFuU#pdMtadl`@b7FTFXScVvme6M4>S>QX%-3hy zF^$L)#J4Yh|1r07?G=a3*lrZ-R~@l-jt5U|R3M+dD-Yt%U)SvH9!jkw&0JeRp|Ayn zy_jXg_P)1z9e+i)0F&piJBr;(e|Ke=m%es7CW_fXJI;EHvFLgC5np#;akT{C9F_Dq zh#;JN}Ic{h*=Zg_c2vsPa56=>#*Hp7QjiM6;D>F`cpU z3@#x16g(c85m;WEeFrs+M&_Jewge~qBL&IToPmsC8Adi3ZcNsG_v`=mAyB{bz0>-U zyqfcll^AK^{*dZ z$&6K=K!&idMjU`Kav-CK6l*?n0;fq}(uF$`&fWR*Z@@0D-MN=%i7KQwIRtFSeJv`O7C^;_1v1#1*n<4`ni{(bWO)%U<_=zhw2d!J^i#&s5m>^`108@~z#Ej^rA ztE+uew=v|o?ni0imfYri=i1OM)_@Wi#~iv(Tl%({MTic*KSvk{6r76v)8AR@kbWt4 zPUM@)9B*$CX;k|U6sLgJ@6d_q+_?5udjJqvd8M3}3ns|Q`1+zc+(r@b6MroP$Ty13 zIe|bw*VE&c8iySY)&4RzY!RIe;V@?DcV|&=G!aV; za6}%Ay=pFa4{i0VvNlyblK#zCJ>Hy;v-F&9^or^>&+i}P6%3~1C|+8?CH0Y2g#v^~ z;*pv0>hHHab>`?I1C`eC2{$G=*^x1bcZ{Q>qs=k?a}3)^!H*UhVkx%V z#R=>wbo{+mftGr=1kb;i*yCTIYuc+BLf7!11o`#YKLIqy$&$6^S zC*PV||8R@nmhfy33W8|~x=CT-zhFDt|J;&pMR&?#&oYABeKl$omN%Ip^n0Oq@43$d zxYlO8N$0duFBx-C+BV!o4(m!U^k@L7uwRCdLv_f}!>(`sVpa|PzR6%dY;}S4Oxf*C zS7w$0apPS1KPp0w4{&z%4vu|%zQ?k+-{%~0?A3&rj8^+j_lU^?l&xGl0I0taPQgFw z8+fj0rtw~zBdiBP&)hxW$DEsz;l5?3Mtz_JeV-b9LrvdE__=L=^gDPUfcfp4!LvR! zu$Rm!SlfGNF8|?IK*7qTGF{4^&Pz3{{G%cAVgRjKdarAREqSc`#FKZHm!{U==X+J; z2mNM7;KEkGSih(9eZ!VrWkSwlNPswV&VbeK&tEmMlCQVkzwh#H{Oy51RN3Wo`%sZ- zUPwv9RxQ5mA3xsgBA{k6hDAT@+p<2t7vc9s*a^UKPs+~y^A=k*$iHU&a0rmG^egli zw=B-zCVVW%4J;R5T87P^wf;TLpNsUXfP%Sr$M^QXnP-dPx&_R5d-CERG5-5}yL{4* z0uS$pO*sB%_+MPMaa17Z$6%XelLdEJ}fNCQb+l6e$HcfS51PBy>b*dq-6A0 zEr5b@z%%=RxNPg;Qy_E?Y}UK7$JXdQydiIfrnllcIS+&tB-8RsAT>J8Xa%_5@}i9{ z`g?6zajIbZd{R?d4z@R~5x;3&_MogsyGneUAc*qbZE;0uv(BCr;5irfZ@BCC-9FLB z05*$KPdl@HiY+2Zz4rv;(f=nyjT?#xq=h^`tP8)%?)vGBFk~gRXuQpFL3gGE%b7D4 zzrn9jNz3JV8a`L4#B$pR7PoAI_CgCYnt!GO}Eag3{+XJ*It%zKwEKC*4{^FqRSqX=_@fXPE~|g_u=k$k#n7 zPefRxDx!~H9C0J%q*m!#XO-VoM^fg+>aQkUsaUZp^tZOw^uR#vEjlAoOHY>7P1}t6+OXUX zW^>6gAvy7dfeO6vRxw#QkgsnO$|WGY9HqXAd)vT47e*wfC6Z;*LU0>t z(+(=j*o6(tV*D9$-VAX4JS7?aLa2k%hI#%)?V=HcANHCRn-_t?ib4%xzX;&^2rJgO-sux7yLXk4{FJdea{V1cg&uxn5%{O_1fp zn(=`Q)%fL+2#Vg7zq|)Lu>$xaX#A*e6-;Hz3KPs@HP&5n`h&Px@Eb(^M0dwqTPY8tvQb^3#0OP-fE@DuH6 z*vaVPFNZ2kO-#f2PjD$?Dk2EM0)^e;`vp-pv5(sBJ*!;o(w$5Un?@YWU{@tO z%CBDR0KZ_+6TWk0Gw&pLb6}0oNO=_qM(MDxcR;H~w8TiDwB*gGhZlPk8^wSnTWY1c zw5&K;-a*gB6Cya~|H6d*3JQKI0PiZM^u0FdOZ0C>m3bmdTN&QCis@aPWA;P>+>d_Z7nK}{^ zF3s7d+Bc{HwzbfmgsrKOk#jnchDsw-7TyTe8J%`qa_<)numBn%zA7y*m1>!m50^sU zAMs{7DS%7wi4{VV)JSX3xy)fKoY>O@6h?-1!u!Gc!aGHCSVvPJ8?-_^+{kWyZSsL) zH4O`b&2hA;es<3Dyj?jtrHHt|#gf}q4qyJ5h$p`yRG%>#M1MArTyE@k!Z(}r7YX-p z(FE5mCUJqLvPvHq*36d9Faa24zjHxSqPPm8AKS1xn{7yz1nHW~02{oCFKmglh@c$x z&hDzn5G<7~t$e6V8mk^-i+w8~f#U%>ATjgClLB350<}zfAJkU0NY5}El)Cr_jYdW5 zQH9kFw>Xg=U+u9H;B>DN+cV%gz%s>&7_gZt>*}P$11K@w`c0ke)J+{jO21uST%uAZ z8``0YWx?O6MRH@f+nB7{=|oWu%y`Hp(51m;@Kuqan?PB>#T`O-B{}9bi%X|Obi-o| z{NQjQ=|{!`1Vi}qf-I_Vwg)pWdsJv}2`qbfXTe)|)s+_a`1cZSFJ~>y#@Q-I_3GbJ z$2SyE`V@;(WE~qQ9%{v=?{tafAKGvmBhkPzu3{Z(uMsV&OG>YzL_UIJt&OssU(6Fw z{vot!^N(}PCtDX;FeiZnY@e-578hQna@rOoITend3ziL4@s$>Ne%SC#r=bV=WL|$l zq`2+vSlBlhex?wo{~^F>L<4BFvsijIyLchxifpN)D2)tO4ILm&AliD)^U;pVx8B#( z5mU%6z9T*}<>6MbNsb75wVd+JbEJb(b6jbuVf8*f;W+MC9%voCSbA05%t&!jc^SE1-7Bw*uOP{g^Qb_R{@mO-hN?Cth9;p?^CF@PI%jl=l1ABSvLy+@ zhHlH%!tThK1%&@PrshngNpJf)m{*Bxli49#@&SCU?+YrN2$@t56aBFQE3G9^O+IcK z=6qb(q+T3gazFA}>QWw_((3GrL=Q%zz1oG6L(?#^3zT?`S?v$)1l7&WduWJ)to0p9 zyfZnE1lOH)y$QDUP$%cTv(yXlU&0nhbf`8;2#9fB>eQcen+rBqj9FHAa;*g)HM{Lymt~&x0qSbdrIfgY|h$Js(7Gtm!Z%R5?UrP?K0c}i$uvy zstHTbWqK~^88rqQfIM)b0+0t$qLykaiqkZ&1|>-zHxoZYLJ;rh zWuiM{5xUuy?+$zAoff8u$w(PziFE~uzEn{{8kPehu)*3(BD~95j`bFo!azK&pv4HK zKovSom2#I++1}6>dCzDXW?+HAqA51~xyGgWDRj3lWoJU)bn@Bix}sYaq+8!dhD%cw z**dy3!a5xh4*f&)B1=5(Tc8MCx!Jca2k@$%wXUloVuP;s&8ZmX%xHJ9PW^a^jg-k4L%aZJm| z@z#n>(}aCXPsDef(WXFco1E&Y+|}_MZMdSY!Btv)%Ph#dwE9O`)7ib!1d{EUw9iwq zJ&y17%7^BMqjYr5;i4iQ3N|xtDFyd?Mava+ms;14R99q&${<}!ZD9em2-m+_N{2Ed zFFo+|+z3T6Hw^X=Evz>n#ql)8WYp=_CF|(n>1PxW*T#&;-F~M^A(Np`BMTQ)#>CxS zTp&}3TN(LYdHUEdtgVW4$B&)sI>n^O<$M)Hs`It)Lck*k=Q~!J*;(&hG!lTfL}0-z z)hOdW9U{aEY7J1ox!k2dwm0?#d6A}J0;ZzFsxF?KYmxHocpx9$aXLf2K2W@O6V&aM|W0GaBo)N;wBMA0K+R zK4lFpi$x2flq)}tr?5ef(aw5X1T@EsMB53Yr-ISJyU814ShhRyBshO*Q?4_iyrJRT zaAK6mWPx;V!+TfeUU%1({!UTCia_DO-J4c(#Mz>p)^d+kZK;^&R|KYz5)`*A;24>E zdo;w#$Rlzx@m9&YSxi;|255-fSx@!ys^}#QA_jJMNCUB|&Y_`}c14!tANogp+>l!$ zsL}k~O%yY`=Xiy%bHjI=-j!E$BLro#h#34H(L$<%UENFcmVkB}rV532G%Q=0Y_uPR zsb=@xODA-gHE7-*$;ij7I}pd-ySt=pMiBwN?j*cO?vz&N5*RVY?eTFS4m|0n?+UoU z=6<4pxjOIeb9*x?1wA5(O_mmhRvFY)*l5xTh3jUg!FAIif@R3q`PcB0k9ujL4uA!C zM#JK)pM(Iyw{YExi7%2dcP4Dec{xu$S(v^#WXI+Cl?7kET2l=eZ=F>^fG`cSMUq$O z^uyyjuE@Zgh)b69VU-n}6OwF-o+E7YtpULcxswsC`PLC@Bw5;;g1+gadx`Wf0guM5 zOyYt5W>c#or`IKBLXXJGz{aO;86rH&Iw^R=9QbLTN4$1#-a6VzCK_JNbu;VjjRk4skYE`gg9Vqq&?F%yP~{bKxZA%|FYGUs+rIA z;))UCp>?v3r+B*!oqXIDOZY`^%uHrvu{H`wT=7`xxIsT?TuCqcRwi*~$n~O!-T0yq zsL$BQ{j~>qL=jwZvBG6d0^iVx47S9k)C5C`@)f0U)`^x}jsl!m1-#0BeeoKviaQvg zCSgXZM%rPJHsJBc*B0?=3CkgkEtsy1b){qwgY5Xn{_$QegVu!gWf`pGE$qS!;HwvV zl$0*LS88cqglS+j)`PKt-$L-%1oAR|-h->F-5waSDFwq7OZy$VUcc;y%UG7xef7(4 zQsxkr0iX4XPr`&WBFjF`N;c;Y0xQDT`pflgmgtj05Ahi+Eo1e(5eF7W73 zQT9SnyV7}c(>I;A60;)BVJ6Bwv9E#c!ZY3Y#AI?_8q7jp0m7-pgQ($DiRw`0gYvh# zc?+e-rr`p07ho8oSHZ`@A|$LG7kCKR7i*OTuQCj|p(s|BXY5@qJCjga|ui-4pARrQZ@jHGbeTOUK(` z*id4PYJ5tw)Xioh_cAG>E|6#vcJ zs0BnWJ1Oq;tR(5yTU;=d9p+k5j$bLQ5|p0d1Z>nRZ_*;v-c)n5zu0lXEBSE943HN& zaZWLtvDRzK+SyEROLqLRoZ>MHt1|Pl9^sl8J=+{x_LXF0wP0f5!6N;ODxlQD~r4IGGsMxhRS8FCY&ux)Q%$;^pveOTN*Af40AA1Nut#V)hnj-UVb zf~#uGW#F7F(L$l$bIRFy9eW!M-}@sK3dB1EIN@tXIUV7BH3{<9h9jYHG1=nHRD8nT zBY|B;-eNozU}5nASO3WQdm0oXYxF4A18d|6Qz4 zYoxRgr`PniwR%RJ?hh{wx5wFz&`_74ARuIUG6q< z)EZ+3m2c5MU*-1&NTBE*Tbq^@SGq_;SdI~n3Y9OJ^t*4+uUA*3v1H4d1a^BM$tZvS zh=!UNDI(7T`}R}o=-7-uBVuI*M5<(U~3-{Cyj?unv2HXANK zh^H%EkQSHNY)WzNdlsf<7Mku|>`bcUBD`Dz3-BR9XFcEzz4Mz%2iINFysU%Snb^D* zux=IAC9vRZgVLH#JzIo|P?=J{2SJ9K7;Q>rz60}1+5Mdp(Z_4(y_m0UK4FH>spV=2 z<}-WLhp}r}%oHDfO6p+Q@`eXAu6}YwO$cYqk3Jb(2@~>`8T6c5?siMXnBhn7Jr$h1hWE<{F?4bC*5UT%iQp1|YSxG=J-5A|U)S*_Nu2759!D6eE(a1xzHgtzO7 z=77kcv>`yucerYG;mc~jDNY;Y@wAB{+u+d==CTa2El(hO#3%hgK!ND_=mcFyW#OSp zw7ar0LRMgNWwn&0!J+LSq-xk-=vbE_Z2juSB(F{R$VyIUv~}%$aM2aj*;dn>HDe;J zv(BihOkciTXr^1R1?d2kg|MUU!=ZPT)i<)&TSwk(rkYLc-is`96=bhY1$a%Jf?a?}md9L+kmNzh5p60W10{{tulOq%Uq zU!!wOt%ELT8sM9s)xPKIauty|WFDxR8HPx=$K$f3&5S5DTEkiVm)s|BK{}d0E{zCr zvZBnrPSA3eDaI4R*}R!b%|#?+C zzV}5KdD?oqCt?5);&Euo&7UKYGZVSI+NkK3NRGFbpy`T*wz1VU1C#mUc(l;y<1{(u6<|gFPsxGO}~s;jZo! zdEfoj7&zQoTx{0Ntrvy?fxT_hwqS%dyBnO$eO|+X#w~~2Ku{v ztDr$i`*L@b2&5wmy!qWUXQ{2R-FB;48N#!C@RL@+<`oH)pOxUiGBQQPQlcsx;bkktAA7*G#>;LERoa%-_a_UqbljFB z#TFAwW{7;%4IdDxY0D0iMH4l8RepTg?1iN7#4?TvH(rmK^>rJoe!DmE=X7n1+&L{D z=s*WlYf7dvxg`2MZR3}IL^6s?RF)vm^TqHC&v-I3p-(3Mg_x5=fnH?vYI z@*T8XB|?Je$@kq>{xic$FE|Y&Qa2j_{8=JAA#YQ5YRkHKg7(}$cG7lM=Uv!)9UxeO z2ZFZ889=U3myWUNlER)xzN%wdKy4lh4qwm*eaW4haUHwxEr8*ijH2ia8ea$wc3LKI zv*6oJ zXrwboDI(kLF>JaUHr!gvMGd>V1#p1{(GB_S6=9mEj#-^ zdXvw&H17{i3t*6qWDb^Hc{DBSPSA!9(3_B+t`0M1AeoTpd0j)S+WzHeliv_tYL1JE z@zS7;dy`d}PD!K8;iAK_mZ}f`JPKJ?<|+mr8K5v?J*>Y)2(g>wX~Fhv;Rj#Vibi|aF+5~o`zd7gI!L9l%*~N4&l;YN z+`>CY$F=t?*IK8I*2GlTzUO#Q<_2_StU6Tre9Ux6^vLFTV(@SiO0}L7wP+3sH*hM$ z29;I^Nl%}Gq8Bv>@2OuB6-eWXb#H^6rMU^g#4}0sFI9pDwi>7OYY9q}-Hl=a%;M=? zxxV4S^>1;|(B5JRkEQV_l@}t3{7*#(Y>-qxeK0*S^T+9l$11 z@GvGof@RdwioM`b_WFhD8eM@4iP}BMZEO)nHp)>A=L>+{)di?nkOcOyTKUG3p@p)M zw|SbzYMv*diOzHlK(Idcg$0j9aGWr$4U9rp0!Jl(=k$9u5@P4gb0nrUU;)h6EG*rN z9WR5dhiHVjrI$@`rpsywVQ|pNuwF zY7d2HTWo#hs|nV6C*B|4^3`v0{JGFT0mu_E1c&pMJ<%WO`ZZC43IGY|dwTCb=du6$ zjwOfkX$8)4C;;%V^kVu+g>7$m|F73IuPym|`c2A!008+9(*RVuYn0GGZ%#IyL`txo zB+pf{wxY(4KS^Bf$lSVBcr5pJkLBy=|9X_87WJ;EN6PrFpXdX76a0*<(sCeo_&BRU zi71<*t>=fWD8vIt$FJ|15d3;#E>_&_woaD#Aiw`Do~;Y%^+dQ zikzzBGKoqo_U#U3Td?r}JPX1UDFLPCyaOVs#NSsFVFQt|8w+a~kvASl0sNx}~$gH417`FYN72d+dPKMyPGY0tTk)$_?BXK(U0 z?*63(e$yZ!@c?iEbOPl%nB4nXv9 zOm+c)$xdKA-w)O5oZ7DRR5^EZFtSMcE!pO3&4p4&kSI0ggS>}wMI>(mOfm>TeBm9L; zaY0!H{eqpOs%ZO$34cQbaYT>z2lsK@NazC+?2?HZW*$O4ywnI(8vn`Flp#sLz(cYoG5+)I8o)g1|y5*e)0$2g?@| z@H}i#PIT#FP^6f(x$jD5(U*q#a+KgpP8sH4(MV5ViEBSi-|pWe@2^VDmA>N*8$+^v zM^@EDrFw%q^&KCQrdhzR02H;xut4a~gI&EErwx=vy?XOP%uYeWzvsNA3faZK9^GL+ zVc?Z7mWy_SpP&@2B7dZpN(Hpoa@d5}>@8e+X93BCK(nK@URy1iQn7(0huv`xS3`qk4hrscHQdRz( z>&VC}^X0B*=^9eCMYR$eA$WzABMOmYrzsek-(pVU@;d1zt=#@LNGev7dzLsyYL4#17 z&_iGBMy~vXfc-oTxNc7|wB~#LThl~U0IU$mi7Xb~Z3btrS(_mnDoEyHusV1>9t5sA+p+G7RU9 z_FZi5vqN8iFJRdgmz{PKsIaRTn}wdD=?1!mJdQ#fcR{boh7EJ#?s@9ox;i3`1!#|6 zznWaCqv&mi=;m((3($St6r+1QNR9w2T!fU`x->0jmNdM)GV>jU&m_b}LsNxdW>|a* z#Zq!P3AIpL1EPF!w4Zgu)nnTtvI@Ed-66rTI`(S~^G+^Z6VJvBnDDu~CYEdhN@k_l zs%U#QfUS*ls{|W9XtUq2HOsni`s$ETHDZ(WqU16lPIcYZ7_tfAO6Qv50{OB_E}_pC z3;HtsRDr)v$^}zdwSjTGE!i5_R7+z>wWZ!#u&?E-1B^(>-sUg4vrs=iBkZ}!Aly3M_#(OM1if+>^6OEOF*rqgjJL^<+j_uF@}j*V#T zPrVS?7MDH-k#C|R*g(YYYdM=lHU&Z^ygXOl?odE>NyB6hrDK6F6l;)Rc_{UZ=$(nR z6W0wqzL(?>b|<3br~I7N_aZ-jnFGK(LuImCPb&fjzHwh?eT|Og`gmfLGk~@6`o@iG z6ueq4S*o|>o2S!x^xd5q;13 z_#Rh>>jtbN*-1Jt+pdTj)NSe6rnDWt4n;f{1_W;;6B2{I{uA91RA+w*9VNod==4?U} zRy}#75hP5k+gZ_t)=G*;#&oGWQMHLS*z#10%<6zJC;YpQ=f(${qdK!8``_~-+-=|_ zS}x__YUZh9)|nON(-o^n+;)kDp;qCg0#1t7#UqVBAMd9p>C+;32~?_gS-`ZmEDoJg zo0(*f2ejF-so>sAvTtbUuxc$oS72ftap85OIqz-LsxKj&&ld1($X9yM4f_JLHN4QQ z7k1CZewO7(k8}-MLXggB_uMG10iF(`_Ei^UM)EmcMQGO zG2@nlk)bFv2gFiNMt#}HTM-Mq1_U%Raxn1H@A#JX=Sv5o3}84Zy62mcLN28?qse*f zlxFpX1dBtY*@D`F9=w%dozMx{X(h$c4X>Y(K@Ry(9Elv?_|wi&Ri!B3C6AVAkZM;p zUH1BV8%waoqkg3KBC2GDQ|D_=a^rC&{$ulStl}~Od>Usz!XOyv#64%GWq7i%f6o7W zl7yE=Wji&*Mq`eHZ)BZ(BZD)~x4FC7E<*2HL?4`cxj1HR4f?KVdog?u(igyLR1M!&39z)t>cF5Y^#<;v^?dP@K4}6`a<*VRb{A zP=^Btl6h33BWX2R0DuBD5OU+t<9PUJ;c(Ab{6)?0XH!$hLMliN{%(>J^MXK3$*OY` zJ<`Vjn!^C0F5AViq?D&-;f6e_TyL5mJGHuLnfvs^*H`>1HJglnQz&7L*IDsHxzvyC zhY#goQt9=vho4(Tvu50<;Fnd)hu#xLwZ5hB8PhegLCeCC*{GEdSJgI4ngKr`r#AKk zo88>R{s~LdhR|C8sunR&@VP>WeJ+_4`PuiysBfRseP}d~gn;EGgGj#*pXJT$n*3>; z1s9L@Y}>meu=*%+dR)T1N%2u>1W-ZX{q0$~x-L{aLTuaHx?#kh0N=4+rjS^Rz$Y!{c zow&2@sSG%KW51i@)S&pljP3_h_o_x9Tp3EEurY=b_qv3yEb-L*@lO84p*jW0_`38fVvqy)h~IDj4}RG-$J`a^2kB&H~uCPP%ZotqHG(eQXp0e-O<2 zxG-sr{GKr}Irt^Fp&_GM)`NiH5&^35+|=qx8Oux>3r$IW#67n(u}E8n$}N@0UUupx zUGVl092CLXO+*l3RRv4h>sxmaUwdfU1|lZTc8bFL0Qj)!LTLSZ(fou$u%4wIyS6K{ zTavX852}IMW5Q^C06yQu7n(YCU(`?-ulVZcl=cBG!^ zDE`zMz8>MJ%yPqc!`X-n>b$j|sw0}SnJdo29X0~r3}SXT5-e-gLQwzm)n0IrmMb6UrHxkcTn}Ne_(-s#(~M2{`@wQz)z^2G3pU;)JON)F zvB^O|gN41nH>g6%wq+utaoWyd$JRFi*s01&S?hexYzbSH6cG64JIuye?!Mfz+8iw= z;;mAfYL-8p!>7pMroVC^f^K1^V6%-o`be4V!nkkFJVe|d%HgdGT5M8Z=SbwsE^;6( zbb*@@iM^f6cR7E!e;=McmR8_CBrcbKj1^o?TA*v96O63uC_G*`#h8t3>rXU_l|s zT*8?g_m$}gC+B%Ig$&%XAndlXCL1_<6={h=B? z_z5(bj5MuL6fptvM@*F{%Lmwn$qY&$kX654VMw(<+61NOobi+xWMhp7*F%=ZY}U$L zR^QQTmYRWC6Bnl@K^G*n|$9t2WfSxT$ z>cn9$!&*pP>+5G*8fQ;vQZ1O>i?##M3ttm=^9PU8-QZe`GB>#pSHuG^Xp$zhtW3UP2${bLJt+7t&` zA@CJL4}4gP%h=v3rsqe5O29JKqm>M@0hZWQaW~Jjs@VaLs@L+qF}CVym#2S*31_%J zwyTj%`y+F>all!azk=g_#F>ulArtBA_5==?lGf~R;IfMCjj2<3m}3sA?CwkncX~G) zqO{WN`%Y!X2wp5N%pF5B;mg1b3eI~iG!rB4 zAm^uVN8KO)P$G}#Q)IUXMg1>+HHQTly3C8sS90~U1BYocs79QA(s^@L$UjU|pPSQz z^bR=dhP$%3wD{g*brygH2d}kEjMV^|^OYi(_Z3Y;%Z@9QSiV#2)?4yuz?4#jkLS(D zWGT9;Ta=bh!EpEA}MYYIAPde8E_8KPuR8YHUBOt z2?oEEIMq$U8eoDji^T2T697Y6{zcP&jZQ}$Jx|+=xH1tzI0b&7ZxAktf+zw3%j>Jv zCDoZWK(O>BwVcbdPY}eQQs@15)?L}c z)}5K-ZQSqm3JkTG2|3yoQj3XE&;Ht$kOlsC3xHYh==`-Uw<34U$B$OQ$ZKYVl+|Sf ztz5(SV`|&>?jOZ4aZmOrHMX?@XR-*w>*|7 znYcXXKfH{68tG_xg?9q)Q zc%6MIVf7u8l0Pb^nBC^n>##o}GH658zi*AUDJkecG@xuT$9_~aSxKHqw6$({3+X$Q z=VeK>=!;&MW{>CaD$|+&G`=*UZdwz-VESUl#gCSA4Z`kTr?nUlK>xjiX9r4toF}Yh z_c8&(H+0@i> zZF-aQ(?EKBTBIu})wa5*7ZB@qKdwqtk&5-|CEM+{5Q}IQyTeeP4i zaJv6{BlK=+B?rG{1>2)A@&t?uTs$eVGUMshUNTi17GZ=KE1sNGV%sTN2macPS0Qf^ zD}WoBZzMDmPLhG6ko+15YB8gmloy9tnE|>1kzPpGNHI`l2oF?W2&RH{`k-i?2)k{F zrhNz6SuPD`A|Gr|4Cr3`O2y_eoo8NC_GvTOxE%)BR$t)2(VKgKBy&v+acgA|Ab+&4 z*Yo1Q)z3RpPrr51zb@_y00`g?2cGurx?`I_Ki=g-tzw`Vp!s>LMDOsI_8rjh2Xe>x zu?Ls`_RH2RWC9g)%#?Wl(sl{n@t+PLt*9#ixhk{cNw)v#LnJU`wFT4fqIKJ~>(8f^ z%LlSkK+BW6K)=QRFxo|`@fOyl`H!itf5`P~DgS4XYfpgV%U`ts{vB@sQt&^W)0~jn z^~;AiuRL16@_k6C7Qn5uK+ERc`yKk0=ld5i-z62YQ1a|5|Ld5;vk=2%mzQG)O?9mB^w&P z<&<%+g53d207xxW5OJ+CXW@26&H9*r&Le^3U##*|=zMqpvmGP%W^3wv_a2o)d5zRC zL*8xD4AeWI5m|D2)0y(zev}m#ECv?7tf}Lc59EXNfH;7}aK41$E6W!gjM9WWJ*Syo z0bu`)efW%fo2A|5^8_&F@?ZXx6r`#QQQohDuz)*pmnAy^>#J=RcHcC()q`3W38-d9 zlGx9{8bTvC1{~DYXayqJqDaD_YcC|UpxaaGkL6B6C--gBAwV7&sI(F39mUmbqp!N9uq2h>#%BOG z9Dj=%670gHu z*Q0EKx*VcjdA2HU3pRQ{%?=eVTRrvoxJp!Fumm6HNw4LO2mP0O(u3lnD8Np7AcX$D=NcjF$YPll~k>q@} z#}K}6e(W`<1Im1F-3pFpS3OWx8p<i=efy<9qz`;oK2waglS~>isgi1yZ{ql90FpA=$tm|J(WLPJI8N_jg|rFC2A5Q z0Ur?CNBLgBW`ZbR4W`!YGjEP&S{;O`f|i#U-I`~KSb_=QWWC}OLd%W;C|gjAed z>3#+2xa|G^*@!X$9>9J3QyNdU{d?OZ0D9*(Q1ew&O{-^{!0lAx8qWX^;C}4>upR!x z_CF~mZIX1rx!_7A!KA=gh?`0(GByPS{@zb!W!zK0O5Oe?*p1y0*C6;h^CidHK; z76uYKdrk*|NZsRouzJ^xxWeHkLj3W!Jmp4|yW0cqatR**g>&ze^fnnwh?gtv7cK!# zcG@@6gM)1g28F$+04#XD5r8v3hLa@sYdh4-{-`dAd~!?G-X9`ZO#^7> zzV2(=wq^7`oEOs|sWZ%tPZBmb@;sDd#FMl4_d!ugPE(x%cR=fG^Sx}yAEm;%ZnVli zBAq$7JLIVQ!TU$APgMIQygz&3<>}Km{D%2nCM2AFuYL5!vFjSq=l7ib&@CN~9_Z@Y zbI9f1o3rKDEY2LHM7@F)>)T1hSQqz`!x?(Y2YPyX2rY(o3;j*Ei64dA@|9q(fU{B4 z)BG?g=0DZme_qXT%XwfU4o&U&-t*woExj9FUUdC9qf%pI<6t;v#7cN(6aTB=#=z$A+x(Vo=IqUL zg1pF`h2QJW$$5w{&!j7jGjwkRvZ-{Kgn{BG2lGSSk9p>(qj1Q$DZ%IrirrD#- z7MYDw>!rXnb4!dZCK4b4W0HPgD&moN=>scYd@T=%EQji$QDtKXOFWq5jjgtG)%~k3_K4ZDWro86z3g%&S zDVq)JgAA&_?w%q$)aK;hVVnzhQ*8=unpQ5RRJ<5zQ*y7Tcopdv*x}CV71)K05XvtI z=;ej?h_KvTUtlt8=~vt<%gOqa5q>qsIZ*X{*GzMFQA;fMHV!&^*yrQYNdvTf+!DuE zgLI8FL1`D8X7$+~M@U`2n+UgnfQqe= z^tlQ2FFSQm%jfcrm;BpDxtl->vge@@O*^@Jr*xhMN)84ne7T=+_*ciJ?E>|ij;D{R z0X;y@jBq>dc=5k|dHOYp6#p~Q@2jU8fG=~?t)6`wJ$61KK!CIIz>R)+n^Uypr;+Ob>B@bV7I|82~FNcj&b{|iq4KaMNEoI0s;^X6;GV1d^$3b;|( z)$r!Ny0?xI4Y|qCx=QRQReSdLY44PgPp@{(sPNuS+TNkdMDKK_PwPB>HeAa2&g=cW z;-ZtUUcGX_45%mB^qkO{ACOQ(8}5x-`IOmRaW+Nm;e%nZ)$$=!)}5@9SyBF- z8pW6w@Q3cau-%~-1${p5Q&0>(#dqu0t;?1R9NGBjVnZ0e2)5jAYQ@%EQj)u;Jm_zI z1T*(pKy~XsZdd$rpY%?D_i1$pIvGD>(dWnT8=R*-G97W8`}RxXgN8R@QQY(P#W&7} zX}4u!u0%`sXTyBl0(UU&W9EeNTQWPA?tFk;9YWZB`L(t3rKO8vT@(XsBL;m=sEm{e z^p&}pd?@T;SG+-6Nr$ERA~A5vpnjoAT}?gNCUl7>OD*koyd(aG@gCRxqhsX~@j&I^ z@*0s#U?UIHNu_FvvnvrgW~~bHO50a(1pA3pwpxSL`|D%Zn@P9R&~@2Uyo0R@%c*y9lhF@{&H)U-$VQKsgWa4lW&u0*wiHXJEF>u!X=)WVy_ z$oa!fLa%`*)t$O0lx0imd6EjY>f9Kz{N-2SXw^LsU+k(;A5B&!=JQtPA$%#DA9M)5 zsri9my}2dDW;mkX-ev~7UM?mrjol~Z{862Pk&PFqMWMLWW%QgHJ5_Q?*?ODJcR?ur(S(?V%mhD(}xld_;45X?h{cg?oL(ELnPk)zDi2FDRT?d?!vo}udnEcCu zuY>#n%$D0+1)p)|moGnAI59f9>=1yzeZ1QabZ7D>9(v@tec-yI7G4}5&84|oQ3uO$Ee%MN)ykq^xMzG4sipH^|(X#GbcT@{ow z7X4nu{$Kq3=SDwWVD5WHyMuOYLUyXAU3<>30A7iM$&tVRvV)p81c2i*ACS9h|F{0j zYr3N+=_Q3}ck;l_r}lXd^xe{J*LsZKp&573=k#N_I;zhY8@cyyzw98c+&y4!fl0nk z^V?YF@9Qmh`a8fH*X}g!{zJ5Xf0gh7n45W-X=r<2+Z_{AKOS|Ux}m(IyLM3SZy*0* z%m07a@~@|)1CzVNtUJT`h(;HHwy7cXY@Lm{ED?3-T$<6>3T*}z6oQIwsi`+Mq%>3N zlIj^B5(Z_yooBAy!OG`Hfi1rOI!)4co{&39^QFyn>025q*||}Dq!B3_(VIV27stpG zqpqy19O^L}f7YxqrOuB~@rJm3sxFV#{Mr1H~m5(C>ggHor6q9WJ& zmX<@veTv()@6q$rNQHmL$or4mSXAx}-P5w#aacx0n4L_Eg#10Pg1|dcH-f0u4@g8* z`v)C$8ab58T@9#SzWcNITXDM1&9PkDh5*Dt`{38XypRVw+^ok!KEU2zhC#Pn*O?Xc z=tCxH)>_i@d%|Db-1Rw4%CL~EzQU-H@1YM@{~iW9fAjH*x%^d|?PNK6K+jlIvN#hY zg+JKiJ@i;P+B5YHrlBcht*HNOC=gBQgs?XcHSE+JL%?7u=Xj2v*{1k}g^n&j0<69j z743^!?Dk~Z7Z@>Bd)OWfcIsQn3VHqdbl63W5CadFTT_~;QFOU*ygP!0hgxI#oVa zN~dD_n9q`LNQF1roA-Xx%KP#t%99&b8Y`@4YFtAFA% zBH|!_4-XN3pVlam(OXtlZpD`U=Ox|$y4DbSNAub^XBbA7qP^chSfeI@Q)m$pEf6Q= z^2YZ?8#O$H=}lx?X>BjU<+}M}79S>i?{OuLndh^_YhK>Ws>OmOO3N?hV=s`zL3B{_ zLfUv2sZHC;cCbW7`CMMeJX9urvzW$Os5hp9J1yc@ z%*s3CM9)B%dgrPX)C%on)b|V+JFBo!dQP8HFu2f&^H7&EYF9DG=%3cn(OH-~r>=i1 za;K3nz5sZ^Au(*;Sxqq9W9yJA|$8`FMW68EBcpNBqeV>AD zLoLtFS^j2UP8$OTV>e^S?rr04W5}OFUoQbXgq!HAp$z}Vl5I1qd#J4}FP+&LYTC*D zkL8j8Lm9ZlcYXbDPql44pr83a5BVRL=O0S`!;?GQgP$M&UkHxe)q5--%@)NkJ5;HJ zMC)pb+ioiIsKlQJP^}jxP3(8N>N$?o(Id-tNAc=|njv0c9%wWcbt89&F0Z3;~8^PU^Mrg{G{%E6@XuC`OB3|#{( zD#HoQ9TrvaZjy|Gb$e#oo`fEr^XbUS^bWqJkq_z_b^%7rcXn)fP9NRf8{;#cBeYHA zj>he|0R)O}-wrX2in8n3;v)FnF`i92o;#W^v+yM};62|{-*ZzAaH+8flY4hS9Qt=( z-1>kx^6t=^!`Q3Fk~jTQ_aoe-8_K~+HlSdmKTn2Gml~jJ7dhL8j&mq8F8M&|! z*gy8Ue4pzutG>_B*#BbhJp-EBwszrdWm{2E5f!k}n^L8tNE7Kzx>5zC2!xVA5EM{Q zKza?*301lT0=g0Dgpv?x2?_xcAVEk7B$T_@<(#ei-1q$bzI*S#1Xku+YmPa`9P=5^ zc&0k%u?hiLz(KQ1>x~8)&)lMRp@K^dSB|}Ma&*!Pvhp~2mnT`$?Ar7r2bOPkIb-t= z<`4AwXBfE$uG|{*YNN81Q_t5i&}!zIJt5vnA+n5K7eIm%6KX zS@rJ2llu~YH>x58T9Q zLWVu%{OR3ZS%T zZF7Ml=VM2LI8YysDf;k>!Vtaf1!alDH4z6|UO7)iz+~4~Ud^EBFvr*=$FYoBnSJF8 zEo7gu_U+Pd#MZ1z#pviFQOFYkAp+H8c+)nrXveQ*O&Dlq@%LuTbXG$+VcX*yW5X$Y zOFFiO+vU+>M!b%>lNDW#MllKxb@VySl?hQE4TtUC{y)rlAuUzU%Kdm z%;u70v|UEUiO7uyWdCH8u9Aly#~+K0h}+{JTjfYD{ap|(8#M>?LeOP&0a4h9^9hCsU>?%~{-GRS>+q;( z?YKXXnmjiXG-}v2|Io?pDQhQ%XU=e+-~OKU`y}yx??=>Yx${;=yFutvkcV!B^kITN zq9T3}=c{#{ZuXR}6G8~0L>76nhhV=ED>G@(REzQTKX3kVc+}m*8iG;+*xbg+JcEQU|MY-A zM9JwBS8_WZJ<@$`^%E8lb@xM;OhDV9eQj}n_Vkq>{HI;mKiUj^@awhxR}&ufUo8jga0IlQ z!~=LMt1{!MEJhn)#ft=6!zi__3{uF-dP3~1B|bM$h=})p7xW#kH&gC!gR0qjE=O~m z`cRs$y95?@rH1S}0oW2lZCqkpyRwI&g(-2V_9pB&3`(pz0vlXjYrN9Ff`#@;p36D; zW!qmNfb4+Q##(ItnVgkoLgAsIe65%T5GG<2>FQw^~BW{P$@EZCKo`C7*_ntii{>dN@r6I_>2zCDBG%-9GKN$ zWJD?+>CP%6J!-0->pHWqBGDU8AarD#!ts3cyMBJA(3>Ayc4m7Oy7p*|st7WrR1R7w zE25$ucMj|=PhoWWfnwJRHLs6R!d!2Ju1ly#jX0mEG9`3D&#XS zTUv!6Fw1u-hhB}nP?ENXp)0Wi`Cr~W%#tVI7qC&s!w`kf#j?-IYXGDVkf4^`fSKwo{fI zs35Q=x!Rp6O_xK)rKa-JClu9=b1PjFYGJDPD=l0E5+lNmPM^*MfYEwIFf?j{WHPV? zrE}7IKB}@{f;S6%Z&v0R!>vp{Px`5P95gUhro9Ybfnhh~ZxqD1!ZW)nVn)h?!(KHJ zbVsY~Gd4G{l%chO?tRR=!O=vxXlY$0JRFr&(3KO$!3A@+4uxIrB z6C@K^t!^R0;j`*Bj>k8eh2vLYXFo2#DwI3l&2XV_lLB>eiHhje=KE{5mj=M*`jdIc zG#~e{L8$+jQdA*RA(1u==WC7h#@il7;InF9=fa_`Y5F4uS*>iqU`y^!TO<_^uNg>U z2K7gPHykup28kR&OYYipzE!3<{a*uTlIaFmv!XRJ0A^O43|O;sn}+4C_8Lqm@&B5t z0EaRHlje6Si0L_3O)AqL2Qt$5^1*c^@1=JaW^y=;t_cZg)0Q_al9gKT&-Po*-zCFr zf%t$e1{k+d8oPjPx=^U;I|Tdr0UWdNV!` z4kaZVAt+MNiE^Z}7#nm|X`(*%@jB>PUy3i+d(Vhkmta?8?7uqg*%OrQ;LhFE(Joq} z`dr}GaP|!F7vgUo$*rxWrtojui)VBwd7&eN4+7MI>03}V>Ad}DThY@)7rOu2iATuV zQV8Nft&EST9xx4c$2B&&yw^s5H(X|*+#l1=-=80^nw4&-k%ciRTFL{aSx#BDE^vSi z^x!m%Rh{`lE**km`F)zdr8j2*cYuy%$Y#^GHo7F#-O;|N9*SAzJFPc0TJ*D9e(hM- za}!75=6NxaZX*^$B?Vew1~g4Bxcc!}?G0Nl#<(IJ4lgwO)I1tZIE(QyW>KWgq;-t< z;NwaoStV}`vgtj(q;o;WtgVGCNfb0LzSg7|@y&`!&o(T)7e9LTrSfeeF|cKK_^yHLs;D$w9!?Vi&VF~&NdFbZ*^IR*32kBK@e)jn7 ztDrUa(9S3OJYzDCRS_a z`s28eV&at742{0@I-V_+en`P2#gkM_LFN20ZRQm{=UnJ%%0Sypb;rw>LRA(DtbtkM z66~_+(h8Y%!YWlWgVi9zMmPLN`xEf0{YpK5dO#`uL`u#d&+^lp0n9%h3~MNb3^D?h z2IS8F?gn$Hwai}ba+(A^!EQcz{>J&YOOA?+yLRw9w8w*KyhTa)kgG^hiq(S(J}J4; zd|6_?k+GLYL1v)3yvtx&($VJXl=bfE(e}y`6<66V(~n6Ro}7+rA_Xy}k+Z%!z&6Mv zC)Ai~f< z+4@w=91AN~JYD&|XqG7HO1Xv7!&9%by%N$oytmN5Z0~mWv1-OGPLS3*tWv+Vsow(< zV>P%VXx_33B97EWCi3Sn( z_0d)j3wLefEiqdHYO}stBss%&)4yCFrwzOB>2YXCpe5|S_dYAIIXy$XtGDF~ybIO{ zCD>9t-M%tO$9JP6$w`2wU#B%2xi97B2&w4cG9BjcO1fPbgSZxiqIo&BZf!?XDe{_U z#lTts-N`9JvTqGIk*Ew~5VBBYq=&`4NMgWI*q1KDydsX&W79Jko~^qSE0@$o60FAn zwKUXDqHL%rSl3q>YHXEfix!2=R+%avS$WVTv4@W~WTsmy6AZt}RGfjJGV2c1#o8?? z(Ho$MS4UQS3#RFD+JFeS%Q!#qXd^2PF>+EM>yxD7-~Y%2{#e`!%(g+X#nypMf&U<40eC)k-S_H^2 zyDijJ8eL-0(~EUq@97mX>%VZ$9lNH$aq~g(V-pkyX0j)xm6;6K>Elt)_nvqzxJ6?L z^B2AmPks%FnD@5q4dKA+&B@(XY%qy!Le4a*RhR_Mj@Nr8?Na;lBQB2^Ettxp+yK%T zaw;vdyP3Z;fsuyq#ik!YQ5;P5n(2PKp_}92w<2RokptYM)DJdOWehCfEZ!)2se+hU zxLv*2=aRu+sl^9WBi54kG3oG!r9A@@$tdAWK@l( zhMtG zB1Q<=9_6pHPzG5anE~d-??ESxHeh>RmRYvuP0ylhHF^_tezC2L#mJr4+zp1cX+{~R z9ZDdt6&M2wqfMR-6Ml2&1dQZHH)Ert*_-fqv;jU__1RaBKWw+qfH_@C8}5Ea3uogm zwizR+LnK;tFTvzwfE~Ne?}l*BCS=vWUR2Uk8^^2qx*y92WAMt?)6F;~u-WqVjW^%czn|KOD0cTX zw1=CPM}XHbZGlI1-F%Y4>fSv|MjrK`z^@@3Q{z*+QtWjlXn6>k_ z_SlR*Sl~R)F-zY13f5L;L_ygX*rhmWT4ya6n89T55gfkl`Zd|dYn4{MNY-5JQoc=) zbr`sAPyMuM9;k~SHnL??6t9AfhGC z11q(Ehz`mnBoxrl-X6jNTqR;zFE!@;4f_~y=2Oh>8mxWy)9{sZu)+HpswT=lm0>L` z-pWRSHVKb5FOmXr!c4I#Ph>ulIIds}(;d?o49I3L07BQy;AH8Q`Z384pW8)*C) z0tIJ1@^lT)6gU@RddGHBw2h1J zi!>0(SF=l8FnuTTl%%V=Z(q7mRXfl1LYHsLYOs8KMJW$0&pK(D8%FwA7#Ho z(Tqw1A?wM#)(gFfa_K^*>pT><-yp@S`0p*XtB}tgTWBR5S4*$eLKz9>&t*#VRoaC4 zyF_t({e+#%*@RVDY9K|tX_;byb0@WFHNS&R+Bm;;;RHO#>LqP4>4);zU&9-~N8{jHXxmyPHLBgJpF9?r?v1 zdIv%_+-8kE8ePI9l?u#ThArOBAKKV#cAeSG|9j)Cf72x(!8er=we8Lq!ESVg7l_{ zSPQi*s1+P8|FUvr;q`&Zwu+;RWSJ)SiY;2uz!|fijX4n2ibp2)0#6ftF)_E99Tb{5 zvSD;y>J1>_)zjtLvhzVH(0OBFWp=!BR+i7^Upa<$wGUY5PhLTv>)hM|$IejAF=iXI z?OGWZPw+_Iw|9%BqLI@!`A7khr~F9B&S8L?>ARJOG`~o`-TmTsU{2-xi5DcUJUE=- zFghgTu!3aO0xCy%M9K@9C7a9B^p(Db9iJWS$9Q0~)g8lWX>z?L@+cF8l<}TdIo?wr_Lu`Z z#G#6%KyfhoKDL6~XwBtE@V8LU>Tqw+O$5TW4aR~Pq1i{wE2YgNqv`36sHAKG+<_aF zOuH~b*tFeg7>zeY7VhF~7v+NVp|-1~RwL8iL`+TPsFU+`*H3*bzxjB=r!cD5nfpI}(~GW|~MaUpF9+OLLLk`e;5>;&@>LZ4E4u*MTiMf9ZRzO}Gp6w?BQRihX-z7}(UmIx24;Eww1 z^F3y^J7zJC4xP(x6)8QU$O)$LzgebtYM)Lzf6@>6eMIDr2AkFiDKF`aWx#0sLn%un z?_!GDSjH7UipEFAIO|fMX6q3F?mx0h?b??RkXKIJDQARQ-#Dj;!~^u>ZXZV+b1F-r z(ixz5&QFbdx=d5B-Bh6-0;S`CK0b8vF(>Krs{!rBJh za;}+J9kPi}3+zR@@1iyg7w1xU*2ZDdfhGmxYeUtWlZZ7h1as@8l$}+yo%Ufb&?b!*ChdgbXpwuOwa0f@n9+7F!>!q_O zOF>St_xEjrHq5IezSOxjk+Nisbmic0GX48EDi($30I6m~(02l#6Z6bgW&Cz9ix#0f ziSYQ${KPuypnq&%KRZY&bWHu2-W&YRx|A)H=Wb%05KGwF$ya}A0a#xJu4+7vI5~X( zKrpgsJB2k~+P(k!XNHwDobk2@LI&$YnM-`KOSLr)I5vsZ#lj2wCQVKR6KjqT7VGCy zxjxHG+?>cCJXYs?wndmz-p}OYQ2feVw%Z--t*ywY41g^QAEqQxsrw2$@Q3&$z35h) z1OD$a!6fr)#~$7^DQCn*CGRgYMW{+W${xcCD_M{=E-_Yl>q8lE6b&O|r>k+!_xeZi zN%{LM^yr{!cu!6XySy!0E%?TzT}pPsE+xByQ(B$oMXmr}=xBmSyK!@R-7~QD9sq5a zfnFMm6LAS$3$<_6Ws}~))i^!vvj!R?8y^QuFHpY}H_HLod)(7BFR#RKS zLB%dJD7_qvGUw&s(6}eJb%1R$VJG$Q{?_eT?Uk*PGld<{(Q1Oj$J|EVzQ8%FOGR1W z!h4W_>cFK1F49nAq_L6W{4|DL{e1Ssr&qko3`?=5g|-J6p$3yuqrMcCwWY~YoT^NZ zJf+2DMZRHcV<-7*s<74ObLPcfb?BddQ|0lY(_fvfn(CA1)ZT5Cxf%gD*{;u=#cBD` zPcHRc-&%@XbMgpx_$v{(cb8f_-%hVf0fd2%fW(`yloW!0kVjK2mo8|mjXqj9cmX8V z0-T86)h4+7w&-MDg`^KQc7@ObSLqU~NFi@5Hpaf#rleD^XCfktVL+x*Sl)lh%RL?l zyS2w)nEtJ?x(6wd)*D&L-?RK}jY0-c97^|UZ8?G9TrBe%4p)}^FMn75`+?)RmU|g zktts$+vIGq4*sp9?xp6X^^#_)Ow&>rFLP%?Ou)61G~{}Yw6U7pnJ(4Zvw6U$?>a&) zSZoXvm=X$WrGFe3517J;Ll-B-r>g{be^m^4#O)uiQHV+glX~_njBFK zq2@p#n&b1WZjV?iMXH65a_c6^M|_cFFni=Uf*lM1rUkb*4wrb8Nrn#EC^EZ}o>dyy z8h?4*;s8WXi?=6j$q5d7{*2y#69lMZ{uvmugxr$O$=x|&?P=`it~8dh1_Tcyc<3Wf zgzMG6%E^9>*kyDbxo{f8M@A``KDb)Jd6k8=uty`iIa{pDeVVs~3+jkoy}Y`1oT?iD zBxi>5pGc&wt{<%txKWjVQ7-kugYLjW=S;dspbcd1jdNb#5k5yN2DP-%|Dm5>J#g?N zEzo6oE+D-MfgL4n z4EH|8i}L~{EGaPGU&R)1T_Ia1nIu4ey z_GUM$#FhRe`_P^Ad(F^hCN86}3^A8R4lRLrk+_nQ1Wmn*tXiR2L8UMm#3r)g-ayb= z?}1tKGFxa8+lvrDHo;i0Dv(S)V=M+N$0vN6$Z3#OkQ36+<;&FRMwJ7gvr3Kkq&=*} z^h%sxL0iEEoAe%(iQZQso!Iso6IrU1i~m|9?{y2fp(AY*jqUSl6)}ElL|4KbdkJ7w z!g-ag?bO1iS<)NcUnIRc9cS-r!=P%4J4KO)UXjjd!>k{G7fF}${73taODi7+Okz%0 zwC6roC+h&y$$8J75T9VY)pw|H;hFnnOW#Ea0UlXYm7?vuO*)u=@eC=gxxxkk8!u1J zngGyz88c{0efcK&!kJVQ@nyv@uvpzwY`BR-f3Go)B@SbP2b*lMS6bPZNX-kI)S#X1 z;6jbs<1xA!E2)d4HI`IG8tH=A3%^R+${gV!AggLbm8Gf0oGnc8qq$0<1D(@xc}KGQ z8|gN|->s!SZCpx|b&jAS$||YzIBi8$#xb}>L3$(BSIoT>qM4$~qQ`QSTuyGihsj{_ zfLLMnF@g1m(<@9+f_}6$=*EV>ueSuPDd;VSxK%5iI5#u@^@BBqr$X|tI?78c`!`+W z88xw(w8(D~70T`+eAyK3bG$-Z=Ou_;B1b+jb%P8NMeSSpdEjLVp;q;XR~19z2c&{7k*=j%p}DlgnM zHau`Qwlu^vp;Do22wdTB@j0X8wehX-j+y1JsB z3#m`@k)+1AvZ!fvhn}`}0=tb>Nf>8r+G9Swbf?*HsR8+{VRDef=?XGO;Xb|!cs*&d zD4Wf)+96liM%-2%?^`~azpr0rQv7yA?8V10vw&(6kF4j29hwJt>|E-kiSW+5To``IJQ`q}a$G?LVH)jCm| z>H)>mML3s{38HsktLRF%n^>xIs;;5p1Qi5@Y%CPAkY!8eO6nm*!d$-7YYV6Q4<;eB zEJ9_PV^$=zerZ8$C)-NdyNoyr_Fx_tbax-N*RRCx^uT|KoAder6i^K}rE@jQ=?7Pw zx2U~&F9*S1W2~<%szfiZs6BtyLXR<;+`_AvNzN-Vt0cdHP`3OTcfa{$4MV={h9|Rf z^KVPRrB7}2IFuj0y4SKJYQ(N9+Xz2g&%T5#?r}Dmvv&0FqcI&?Llqqg&kr#keKr`j zH)6FdZQXoZ#A;k=sJKO4b!9t^L%7!jt+n&bA}v@1@j*1zRhzevelEF0(FF6u!P)2^ zUb#CoH6P+*Yacz_S<9p6AmL|H|3pqMo!bj=F^pNGEx&lqkb=W*ygujmEKqkoeh${3 zkOeg@AahayLm?t_`x2(`>~+Q;$Tk>DuSrI$!_dqj@-^lf#kB1fY#thHx1}z4B0-`# z58}8dFra7^x|IJ4Av1i44=scImFAUYQfiz8Y(vvNCnHVE4jxg&^IMkHbRu+ncU6e)Uu0tu#=ENuu@5BWRVaOw(ueE_`Ty7y^@aZ zp!g*c3%PD%d=N_`F9rGiR1~?_1f?8i&jwB>WzPA73%^^9v_0Wc3cBNQUP9M&=l#%r zb8^;A9oi}DBJg>XDH1>1C4T0nO+6K~3tIqW4v2}~eW{>1ldI?&wpflPm?E>=#LUJG zj3RPHMY7H*X#0MTFG8Q_<;v&}ywPR%e@}{bn8ZMYz_gc!m>2?}Wj`NcsFc+R5<}y2_R< zhN}4_WpGpBFnW~-6x%M=vU%sgMJV^iQvTxkCH<7cQHd3c&)(^=v z4Y#+qm*(7Wl~!zItZ@{>m;s2xR%o*}iEWc&g?o0<+`2vYbMIsxS|;V(?@6i5hxveG z7Qa+Y?mxa*k1=X9_d~vqlw~Mk_8G7k(N<|x{gjmAIXjYquUs5^O*D1=Vm7Hi zpeT(k#aV)@C9WUZyP31^h1`I=_orjLCoCGh9ak?Jb2fjeb;)xi0}tICI6L!_lvc_e zv<_|Y{Z3+VD*0-P$2fJmPm{&R6XMIBh3q0s0NmpkR>7*1<~RiCzpEoaOD;!^j0`an zCzz>A;|-oN<00!`<>C4{xre`Py4vi}u;<}6B09(FGT8F1b0YNmy1SG6vk=|=lk)e{ zzqVD#g!XfXSh#~=VZ!J{G;GAMMM&v6I(r0dBQHK@b z8sWFM#O6104{#at{7OKs@1|+4Ox$FcN=SMGX@X(|ME^iR>`Y@eD7S+PYIRK3+UD~l zH1DDbO&d>Va5VWPOfsL&F~cXfoEmg&d-epgls<3|o97yut8GQ(M@f(@HI+sT7r*o$BVvg1B=q(+&E{fT4QleefLSV_|Zt)iqaYNE!-*ML~M zn$*RGOKh9yHrbZtGG|JG40^dq!sJ~E;J|Zm%GNZMAYZ5c6E#RM6_nU95g7aaxce+@ z7bcrbsP-xHaN;)&u0Rxhoe0T(=GvgcDg_yHafZY0S9V#P;U8$5J8q1!YIg0sE^)e? z%jUkGHq7!GK#8&@H!^Px_jq6{IK?S1Go2G!sNJi{2c4X0)9c?K4A zoYS8)#JUoaI2*Rt@L!o=ugSJg7G5k#;&a*-X4+11884 zAsY|OZ|(%H3L7Eyg3sIm>9llLLr}VD(p%8RrI98RRQ;g$++;)NGrDpKxXN1WY6SiP z49NgAPI`u2&pl8i&b=UA_P|4SL{=Hkb|4xDk}mXEdbT>*EHK3VaYs2SR?=mdF82A) z;Bwccrw3U85+7pLVtyefJLx8|Wdhd?zK@$rX?CjUaHZWd+7ceo%xFHQ8Yo6$KH7kI zYR*NA-auJlP(7Fk@JciL1?fi)VEqM}1O^~&x=bryaQIX#WG&V=QM2hs*(X;_0=K?v z!Y|fmK}j6qZ+DUeA8luCGRE~>`H4lwv2#L%T~5Z=2dq6kB&A^xqL|ViMruByuPo7k z0GlqWvHK-N%Y$S>CjlOyfnp0^`ZYb}8kGoJiSCIIxp`(h<4PM)*zUOfo2)poLSPp9 zrNJ}%@MC99j4P&B2@%;;FAM+LMTULzqsQR{D4x$Iq@rQa+vE&M>)@+_C4u;nJ!V)* ztz+ezACSW{WO<5Asqb4VJ*0JF>TcU#(j0bXA66xDO@AdgTa9HPCeUp%U!rt;fbFn?5I5m|fu99R$7f1fSvH>)&Nt)~otm(dZf6f=Cl$1JN>8HW<*Skjab+n+w-{*gqZrPw9Q#mKB8 zWEviYb#C2+dym|&t2>+!X@c`xK(010E|#vo8=K9F)ODkoR~9Qzld6Ys#VE+^-XEe)>aQ+YPJwibAPDAUOIG+x0(S^n>YJ4bcyY0^)ns z#IA2AH(}zYmmF7TbqveF=?>10TG`u^K?gmfTWS;k;GKtdp7k2i=ed4<6dN~md1|*1 z%;=fp{=;HGUBt4};vZDpr%AuxsO7t3cmBQFv0^M=ijILQh^tEbH$Wlh#feY^hy;{R zhS)_+ypFqKaCCoE(uZB51}rBt7X~SbM8FgX9$~rdq{D{ycN2D@nx_VUD7aYt;k%4S zo@2y!$3i<#0mbp=*k?nhm4Uj2c1pT|-S1a+p79#O)e0&q{%?|rp36h_XiuBJ)98U0 zvoqbL9v|lGcip$}6J_~tS;Rl-!l92&0+n7OX7Q{4;1B;8Yz zBLDU4;LU*gPwnj*JsaL!e@g@If}WSp4!CPhCJR5xMViN*82_mo6%J(9%P7*nTZsaV ztS_GwutpsHXrL#R{L^(ohv(HxJXA^b4!78r`of-niTRP-J#~uz9%Q&Kmfn0PJPH#7 zsiZ=)3`I@469q(g?w&rULBl0se|oSR*WN-xNl%>_En6}D>Ho3xRR7Cbq1bhePy<`} z>f=8f@gFH;e$lUJ(_SX|U#RH}8TZQ&NNjv=W4nUKDTtZJ$ZsuP< z{qeQ`&1s)%_ytIZ6TkeJ4*w4$EGinPT>R!;z|Z%mYx&31{dJ3VfF6(jv9#phMJ9jy z6#DH7AVl!qN}&7YUqAidwFy0P?dGUNfmKPOfRV-B#7i-xPtlV9?bS{wgiZr8t;=7p z#pkKN|>rdcc=Om$L?4r5#eVcV8ZO-3lDV=;?v}*QYBc~5sW33!+J5W3^^gDj&EC7i^ez9c219{T$A6U}d-p4#J#j>W<-c}t?rBZ_?q@&0 zb?Cq$fP|-2VhZQo)o}x0`tTW*Bq7r`-ASeQXOV6wUE)YVQRXmN!F*-$$_9`(79Ngp zRqZ+J?m#WBT4{?db@*)08bP`~^H;#}r@>Bq1zz>)=3WDZAA0YDZD>ZvtCX{X{Ocjy zEkd!qnIuo?dgg0@L+=uF*QiIbb(>M+;KY?6D5zpt>g3+zn=_$b?keT88F5VaIN@RE zg&$|}rQd*#QeQui8}P$GQMq;`XtRr1?N8w{`jl`%Dmm(mP#1Ypp3r~~ky>c6t-DP- zyL6(fNM-aVOW5532FEL_N4ji(zmJ{eRC=j~R>Ef;NBH^vwecidj#pg?h1!N`*UZbvd*l~ai_Ly z_53INDl-E=kM4>Z^d!RR?}r-!`TdfjE-KvTnykPm)h?JoW4|UZul*Sp(1b2A(P&|M zgl9s?8!Nr>MKcK~rNqyCGQ0SV_9TJ^C_=i6yY>5y)^2bocs8QF#Yu>aeB7a94jnP> zD?kRQ2Uz&$m1Z4N){r#RzIAUql2-(h0po#O8JyjU+V&b0bb8$T2wx_!qiyr zU*YDTJ#^{tZcl$zBP9OVWuBp*;$IF;eM-3Z)__GFG*G3sYU;iG0CD*Fi1;{lZMq{? z`{gN~dVm}XSXp!Fb#QMTwJ`l#-D1zqG`QhD?4*uOIJ6!D5W=L zq+B-zjW#wqm}O9&4kFB`>&%39Nua+^_bDM1e4Iq%E}PL4R3nERKPL15Df>H8_dE-Y z&GulO`Yi6=z8wanI}1{%(xEn4DM~g=V<2iS1mUrMj_Ccwzeb0}iG|92C8gvSFT4<< zY53j0-Td#LP6Y3LGU$Id^W={n{At4npB|WW>G{kroxM$djp$ypvdyt9>Qv4%-ep-v zpn$6|Do-O}_7E>W129tB^iIpH3|l_uTK1cvTyr;BF!jae<|^w@9$gLCD_!NyZb@O4 zT#aG(m0o342BQ|4z^Svdvi%#Ja_Lw$a$Ydj;$pKKHLmt!dNopl-u?c$U<0|lq(}}#>IUFZBSQIu zu)4^Yi^(M>!!zGUqx0#GTlu#)qN*W4#`HK)XX1n}E4z1QM)8{srSUqs{zfk3jT8J~ z06;Z)xeJK*h6KT#PyHNa6s6b zu@zgxy|zJwz*VchVO_=H=50LIHZ?GRYe3OBmQ&s-O3Iz#KpZWq*$A2YWa+qAo;OqS z5u5tZFT_YuorQJjDy#C!Q|4)^gepkin8j4DEF|7MJ2CMat|GEGE1<(cHfTMLO2Vpr z)_Xs=m1(GWN9?cN(4XOj3ia?z&;c1RDpumh3O>8r58Asc?+r)@2$;h%OCwrMcBuMw zV9f~`9JKx1MF3aNXn5?cti8H1m;Lg7hNoE&55T$+YruII0X#F;VSDOxHd0Sdr6}8$ z6?$!{-zXFn;SJPXY1b5r^afr_7Iv9s1m?fia}gXa>W|M3*)T|#^q5sb_4Xc>CCu;| zmiW5BjjO63kJgXwHwVRj0d}5E|4^X=W-;~L>lU|ej=$Gus~kPNINqw~)R!i@%O)pb zhV#`ebcWfyx5GMC9|bLc^j-x6FiN!#Jw{=CQN0IrEI-3NL6c5J4}X7bM#@|S#f|uM zuYs4w$YZueSta183LB&$b>O{-W&N97mO#X@1;U17G@&gP&o*f(dHM1!UkbLYOqlJo zC?K$VijB>&`x(9XE66{6NhH0&5GAa$LtmC9y!RO@_g)^j-ACDqs=p6YlW>J3rUE(LhjlG{>F863S_W!l`gV+13NB>_x{#q0wvb_flsFy`N6;Yq@~@!2t36pQtfx=egd-kf0LM)Df~Ru(0obwr6nM)8J* zGs7q)!-IO=-={};mk3CfyA7g~JV0K1bGePaGNqa@5tt}otU#~(zTU0qO3zEy5fEZw<&4+K*uJqdJ^d{oiaZFMvMkokQIsvxR@rE?TdDkbyxv5# z`EAW7_>yw*hoU*-mLRpj$Db8s!RznJOmDTWEy-!FI*H5xEyjQPbSF*1=i^R|Klif1 znT58F?9+RnKj*qc9oT=1$ggwfPRXw?617jBcvE=k*RV71`2{u4oDf~Al)P0DWmeT} zbEZlLEp$oK^YRJTM|*ygm-sH+lrdT3toq^kX8&A`tWRNWKIDX5i?1q72-Kd6t8* ztSOu5HLyQo0aGq^qa@a|m3cSf=y_3l_197#i9s7HJr_!o1MuR*)@CL5)z0?WisE;c zUtg4sPbGey^Hgax{)J&|7~g6)9^~|DY(@VMD)V1n?O%iWkD=~xJTzJ$Wrjsc)q6B* z$LLnK8rL%hutQa>aMrVZ8}^pPVKw=+IQ5M|wLhLK5WL85xm6e|~w-Y|F;#nYeT*QmUYzBbb*cLL>Fma)_lN6bSX==zmSZhP~vlR|{hpXbZp;F2lI=|?mx zdx86Mp8BdctC`Qsf>(}=OKWNIISeJvJzkjqO}wDH-hakJnw_zXSs|o%RRBY*Gnjl1Y%H=(&A@NtS;S+ zlTvP-m&|HoFWQq*dd;L1cMTwa-@y7g%9xd2rjaDyA>Jqi=3aB|pAg@{6y!dOT4nFQ z(KGKcwyDgfvKXMpK3WHf1P)cU4~s%P4BZ>fbL0EFQXrz4-|IOj~_-IH_+G zA6s19EbjI>s4FM=8)j=^%9%#0n@JiqDlqLstPq`dPyl%N>9iC!{4l$8=-^l#sboPe z-fRBt^T_a*bQ+Qqk#7DD{BnA8<49+Z81l-J{3K73;KYN^_e*C*1aG!zkJY<~0W?1S zTi?TZ2g*C(`*2`hFTb0=zH*9e=#}^%P|gohY`;qLm#`$mq_?>qO+9_+O)`$(q}Rj_ z84va#XS>t1XZy0#3VRjv%!ki9nSU|&DNJAN?9{AdC#abvt+%dmzJAp`-My+CK>un@ zri^Jd6;vV}llzgr#Q0Z13Lw8COm<~;BPo#9AxB+#FLx<5H9+%xF;6J1`h6Cn{-omS znoV7Z3tkCcBS2f!cH=fcW-b#YipoT1=Bw;o3yXqPZq(mD1UGCfamjm@4)!Yra#I=< zT_;;KG?=B!{k4Zb%_U`>ZYhL$HPl!kNXFV{r}2k89(7og#%xtuZF9Szr@UKLH5Brb z58`lkl{p24or!uj08t3bKA|F`L&7HQmpF2*|V-DZ!%9d z-eUxLR|}yOzm~Gny+d4zmegNeba2vHBMs{D>ZS@RGJU;4E@n<9-j$W~v$x*LQPG~o%byW?iTY!`q1`j9t^Gvvg%z8KFe=br-izs=QOLQ8qyrnDdPJ6PG`F1H$MiP z(LW*svE`CY=dary!}?&!=%*Q_xp&tW+K@4owF!RUvks%;1c$Db@L5?qyC|)q80$Wp zqGOm|y%5I~WBp+{-{tpau`PhOgq4;Y$WVIcjhv;pXhoQ0bKM(@bQuF1OF6tTwg*d! zRf;L)$bIsJaL;0gA5hK6Gz!&intpAfeJ7-BIuy6Bz@m;cQgDaxQkogOy)nnlD|Y2d zWNEL<6&^zF;Q^a?+mP|MdlA~!eaB18ku(z8-#X#MQ7w_mHbdr*#}QSD9wX4ZbQiyQ**wUgyW3ugnF zlQI}w;sM98nNio5U#^PJ!cSI!ol#i{Vk050VJ~BkbBe68_r{;~uS+&^owfH~G4BX| z&%#$;W$xSrN#VEoO-dc<>y8ywytX|B-5?fGFyilD69&HSKXdNhjh-=vP;|6VrPFg< zei+;{V>DgoyiCHC@aPgnOTNyls_N+|N59fy^L(JdQL=4w zJJK^BsAq9HIA8B>d9C7c;An1g<+0iOTWc4Fa<+oS13q|}m*Co0r)^D{sb`b>civm} zXDJH#;|A}xY^-S$M`+s*5J_#WWl5gz4XFMPc0om|5t~DoA!_Qqe zPR{CQ*~%Y7Z-?`HD2`SX%BbxDmNxWuPL0CnFAelxr1He?oG(QR0}hAlg47+dE6ySv zKia)ty~&R((1_knWYl<%o-a*rZuNWJYuZDp*qTxvs#d#@Cgqcpo&>%HR1yrLiChHaom_2 zSlb-OdbN1nw+4EaO89C-hcjd0Z&&9Ql-`(%jHja-eY#eLdZu-dtmy4RcT4H@o{=0ccDehR4YxSuYm7!>jL+1Z;xNmN)Lh*zM_G7|W05 zmw#*8iHp-0h--a1?--0^4ROQ;&-;ZR`P}*SYGiYhe$PP-O<3oI_vWYXl8$fQ7@UHK zb+n8Xz!&*c<{{1cLf@&6(31QvQG@3EMdf$+UWATO z>LH00g3zTyEI|ii_OT{^c=X_^>re69KfbE)o_HVs;n5@BbIPhe&CCDrwW0(_&y;p+ z&;8Fm@$02Hj{ViX4x37?-@E>|9{ndJT+W&L|_0WEN6vblM32^}UOWqj(t(eljq|Kt9 zm$$MG=3UwQ>(tp_IyG)MDd}Hy_&WPbm&Tciptt)we-r*CF57Nm^TQVhd#7jiM$a>j z#bjUU-Tbh>^Rvh=aj|y2L2qw%{?@lA-t0_ov&u!AZ?h-5OV4}QTskg#!{_Nt-eC_V z{k;wkfyVW2ct9U??t1y5%W(z!@`wR8>Hp}^p8{tU>C>04rysMGi~ULX^~Wg+$dCK6 z0T)iipxv|L?aLC$K+vLJ$|3UBbLYs&8IGT@vL8a*&;_-(44t}s zORUXGI|DU?tn1#4@j6|8_R>N0miv`d5$}w!Q$JE|BJo%j*2RY zx<@66h=PKEhy+DIas~kj26B=t83Y>186=AsktCtXpk$gPG|*&FBqxbY&N(WXCcSE% z;LIp~-+OEQ)_Uu``D4~h_oZ*$x>cvn*=O%_(x&?IXm4%4W8S+|)rKfTupt6b2tIT+ zM*7WF@#%W%DqsQHT>@%yTFi6Xh08=~TzITj;^}B52%xl*#NjI1C1p;yul)T&+K~qu z$aH#az+m*P-=gsD?OZgyZ~>3QVm+lgi!WCG#koUHx)v z%8~D#WUD=)C72TQ7AEIMe&i?=l-)>1tx-7@()|mc|JRbC31PPpiQ=DiY7Av)vfVGS zdi{EupX8IAjEuc*1e=D?37ebE|COYT^0PmaN;Bx}GsPTqaOmfnTahy&$VXreE#@i!M&+J4~%XwtQJK4YG-8OB^ z0F`Vvc4#ogWP!Z{&fY){rnE+JRQk@k?1mqTZee94mghKFG5)m^7OzEX^U>Lv|7X=rxJbeCOds*XJOH)yUD5JyfVUuwMQ7vm%XawQw<7b$>?_E5SD!%EOa<4LFdS^ zHum8WxuD165}QL09MxcWV8#QDRMU_V<=jfSn6DujnOIqaMONXa^+C?7dZxH;ge0w}_e;@3zxk5M*` z|D}VMmP7lZywf`>uHz+)5J5)}#V$l#my;0vn!ZaXvmh#WKvWp|`7?*Kbje%?A=z=q zmYDjv``pX1s%-9p>F=yZ$QhTPPZwB)hB@A*mv?O|3S?tZfn|lU5i_WyRv0W0T<#o< z&8BvHo}= zS~-t}^3?L*p>)!dj?pM*oYNPepFY*(h&V6ps5n-uQ8TMoLfUGWRQXKHs8#gilO7np z%XdIJbDMpJWk!7IfvdT!46&UTaA?}eV{aOJvDcI%EHphrpqt{7i`z<6x4TXj9HH+f zKHyR+%cKEIgsk#Gqj?t_yb8H24zwwvPP~sVbmI^iVJXpJDJn*>$?@FkCQC?wU}n|_ zQHSEHp1VPxn%)v4svqd+Yfgd}r$ix6TlEA_?Iu2aJc3%zpun+_E9a)ZXUM;b3jfo-8LRsk0Ng@A zd9V*xB2Hwr8lz^aJLB5!sws7f85gAceq%Uy1m5!$M#tvQvfT^9vZk$3=JvF-v_0|3 z5e$X3M}?~oja8`->haR6A~$6Z^aaci8-<9L&Jw{Hp0%&t{c-9+l}$6Sv4q3>F9P1U__2xovgo5ti%XnA!+<~eBn}io~YQjQY1Cpxj3?8T!#!P$xx24F_w$F=m zJn1$%%fYhd6~}u)ATu&o9zrWgFu#4z<&_4vo4ey8vu;^3o219#K8|KK#Dxu2Rg!wB zohcH`rt8)u8+yd3yRyjkEmUXgagWLYv3dtwHhb~|hH$G$=48C9k~tEVscu@K&zj^R zmZ}g<>qe9$vcaNuK(OhsV6+csmkAwQvGpX|SZf+~AGV-pudJMtH%o`i5?$m~o!?W+ zddLaiS_=~O%p+iN??LWiMX`AQ~M4}pu)@7$)a+v?fUjN%OtiB_*+LM75ojD>W z)RpUw5b1pX!HUatg{MCaLWUB8sb#a(bgz|-dL9*K9emMMk#8qGi;wTb+erSVIk}yI zp=@o4xDsW{z8&E_VEw7#7X697Oi9tG8-}orrLM-F6x}Gb#)`@jA;+aI;%SM(rYz3Nj0ThAnEphUmAQWT-{D{aI&aQKE)xeyE;t| zwUr$0*|e4F?T2 zDK0kxIw|Th1A3>OPAhDadc)Z%q?U-B&xq%R)$~!^fo{HAx_`f( zUgZLr#;r@Rl!rkug9!`S*_d)I^H*929rbp+cX-%xVxSz5qQ z&hBE>{#grv($h`&nwv8$*GTN>it;-TI@gJ)^Z9m@HfwdA-E_X2TcK`{rCg>{rS?6_ z8eP8T5_8#*AQcU$)zw?__x$|=oqHIQpxV8XA%4Rq5tGrN-_D&HlP&*Urc;;b2?@(a zNvuR7gHf`oQU^qIF!rcGnMEUHd_mP+K~W|fy;sgL@|RSuOPD%fGd|W1Hh>9&aAcme zcpNI?Gp7Aj7k@+!QO){@zvT)4WZ82~6m}=lZ?t1CNM3nO?y=}^_NG+hVI9kBik6of%DIScnuBK1gS@C`>Fe#W!nAZfB8g7s!&d>GpE~z=yr2=@LzE|J+@eCOv)KltQ?I&z zlz~StBIsDqqR%0#HPoY}5GO*{*l0?*Jx&$dz3n2pw4$5%2oHI2v$N1`zJJGe57aHI za-* z+Ii`-rn!19?Tc=%GPQ1&PTew>=yzU6HT=pcGHTpdj)O(4L>fiqmP*ZoCmqU@Vpo^@ z#3ceAH7!}duzGvnVO5f`#YPaDAGIq`mPTnNo>ccSWaw;ggrTq2h)?ra`veVQqAo5z ztdw=H`Or$V$?p~Au#D6I-WVipvg!=~5#ON<$2x;iLQqUaP_A-|M+SCV4eR?iE@xZH z#bb3RXIdI7NFghi+PaDb5c%d0Akx?_b*wkoGx@BcPuIC2ZP`N>6EivEECEs6469nj zuCHyES;s2;7Y~H@VoWqn~T?cRSDlu_KO%f#2U-=1wVnExc!~%tCZ0sKByA0@AAU5NZ*}wJhGHR==ftdE0PUJ7Kuq z#>GZhrRUP5Ajw;+{>1E?Hy=IK8sL}15eRBr(oosHRmi;9TM`kKkpcXXM^=rP^BomI za|crGus?Rgt6ii0S^c1b(F8y;74!dPY4^Z+(b|AH{s9ou@F6P{d7N ztI(3cb7j>ge`AN*drc{`Z~a-i&UR4*v&piml+P1!_SZX)pjY#1RSR~~gNQ~okD}q- z*nGHzp4Ffn+2P43)LJZ0`1aB;E7pq*=m#w7aHvZmuRdFa$Yz)sd&8{0M@VXAu7BS| z-tEWj5&oW3)>|SgDZ2Ih+I|ta%T1_5U)QzTnD&w!=^fJQR)p+ZogqxT6xMo{>{p?e zhhne0_I_yTs+lLTR!Ct()-2_Vb1)7)>5$2Ot@#a=_)um@T24M%xDf@+2fvr3-H%8I z_(Us3k>>U?84QgOZFoi08oh2F@pQil!|i$@dFKV$$ia)JPi1`eaa!^#M|5|X*H`W! zm~}WL2&e_7hM#Hi-MGDWcZLr`X7yu3!}V9HAB^;u%NJ2)fy&zVQq$i~oADZc-g^&a z-fBZ4an{K~rq&xFcn12@3TqpcDJ2e)r~-%1!_}Nnc6|)#8|M;o42>Td@CYY;mR+u^ zt7{U;)lU@jJU@niZK~`t|0r9f{fqYPiY$0;>C}%@o*%&t4HW~KDlQGUI|_}6ffD4$7Tp@xOZ>Cy>hhpv^;lam&TNW>5tv=;dF~ok zPwV}_i61QVmuYBa{KIGH!5QEg`&Fm&UTiyM%;%${R(VgQb5ZK9jJN>FVf@qKop?v@ zgEfhUkxAi#p56Se5i3t!21|hnW+7cyJPP8jo;LlYgQ`?rt!h}zUY#PHKtV>c)p{Vg z`gv0dj3GaO>zPxG63JouH-dbu>S% zlQEbP+9^NV_l(Z4i{V9#1LBe{vuqFFwC-8QvGt^{`XxCP#I-hoU)xF(7q9Y@mL_hk z1iV&dn)7n4osaax$tfljJjF6bY3V`;b;m7Re$wN!8dhlWWmXF{LT&wRfks*TXn#Z5 z>RiskH}H~kNve|m{DR_N2UrcuBkoFzd2Te?rZbAE>BTs_xVQPmQB%{P#;-LI%TMss z&Yt_=UHw5ya>~f2xx=Np_%|jWPKI%>jcAsz>sueWyJyzZmf$vL7Us?=^*S002Q3;r zft4DnA5^;Dkmjb4Ve3rbyj*l1Cfr}F&Ao%3T9YI|_{+{Fs7QC3CtCHt(Hl~2+u*Y?l0 z3B5NbQ!RXwsIIRz7{)JckuhqpVzK#w{8x_n7zzN5Zl0AI@u$%_m3+*R4|1y(+qiIO zN8wslg;I8k*QTW^GrM8)dE`jCv!|A)-7oW1#g(8dwx0fKXd0U0o*({Lj}IZ0@&qC< zJUTi!<3O%!aS|5qDpSp)?J4}|n{MeRqpH61xH=sd_xn56lph1DSK9LHeS!B!kK5mA z!|!Cn)m}Z$p!?Mm<-^2O;~z4TY3O+!j6TmookBVe0RzLx&;N~4(!heL{1D=@$#&+xa@4A?(VtL ze}GYvWahBVLe6Rb{<1)Hdh9^caJGk)4fK&R%F;>zllzv#Y_~fsRk!;Zj43mRF0T5hd0rdJebYav3UY z^gE%U#m*ATa4JGw`_(k8^~wEimiR17xPF6hQdiwj@NmJ>;1iE99b}r(asFrq8f8%YGf5Qg(M<4MS5hO{T8?%sXsuVRwc9t%zII@G(=T_sO1AA5z6U}?M`sXL!EeSm@AjULCmdN-{_a&_1G$_WZ8rh=1t4x^F3YTB?0R9h?oiuq_Mm;eY+o-!!*Q;eSL1;C>P+6} zoeqlN2c5=#UJ6lJ@N@R;3mDCbv*2KQ=lceiu&(*@I97}2`YiL?C$Ua)BxPK}p&L-Q zO_Krt2svHKyTe6RnKy;J3f#9lbK6fCaoD%i(0hLvO0hSELxER`CcbPGpouT{H~a7P z#IU>|q=stD%0~c^hAeRUZ=xKl(mn2|nRzQR^j9t`W_VlB3Ou^5hoi0!?MhSe9RM|Z z-n5z5Syz_tP9G5VHUMdZWwhkosgPTQ!dKCGvjqPA>oye?${`AIfaH@*?Kx#9;Djmw zP!PAgrgP&Bmlc1V@?gFj*`~Wh7hBmzG2_36_C45y8BAmeqsvGLGxx>z07x9K7qODdEX zd$#k}94mFHohK0%X?ul%+r2qT&A7Ro5!20TAuu(YkRm2nv3q58A3-i=& zy;g0pB_@ZNvL>LxF_-uFWw`$e=;X=m^Eq(a@eK%!q{csTGF?1~Gt+o0W+*Lpb7{e- zo`7=5(lSRQ%cC{+gi9{s?zgYHvz=aFyDY;o&y_NHw<^gGX4#3)JvG%B zYR|Tt<^RahzjX8p+~ATv>2kpzfq1xQuT9#3r_7W5ksg}3;bTE-fFqL)chbO2g<_lj zm&-ARB0+Oq=sLUV62^Xn(=z^m(=@8dpSiu#sxY0;5>Sz+e)Hyyxnk$*oWqHu0T&)F zrXc2aq4D7tE+thK49n2UEvTm8cF1XufWK)j<|k7Cn5MU)-u}49N3w#&wxV=1MFq*1 zXgV^Ixy8ObID189bAQ(uDU#?WFb)z6nz{`gM)lWtcs@BTGrx+(b8(5H;rXwOd)dsw@yqazsD-Rb~HTP{uJgnJ4`S;BIwME6X!@bo836f&Z zx9v1$qbqtE`huJ*QDk!WHl{P$n}bBtPTC7I8E3w}N4&AM+0b%vGv)RlbY~6e>eFFb zri*;oa;$i5KOW(KqLCU`-)=8-W3;N-uqk%QZ(DmN8~P3!w+)$rT+Im2w7Ntgtj8o$ z%WJ!AwF+8DU1^cONmOQN_)eXqF$3z{Q@=Z9tP$`2QG9A0>C9a{5yfGh?|*6i zWwdAHf2c3!W<%Yc_Yqd3R~M1qkOgWXs9^JEQqU+jw-^O%EGzz946}x{tEtn%TFJY2 z@9d4j%^{6ry5{;_O0)^k+c@|!TW;(s+eyyZZuV1Bi5^8O#M_pgdO!_QAy)joU)R{` zx%ZeUz5SQ4FjQopaQ`=Z66}P%kzf3w?L?0i*TluXqx0~o<&f!k_uvKL+kkoSZYx!1 zTFb8&c_0I2Ny7B`IeZ0a?AP*MUM#WLX49Z#9?vDr92)JHE|WNG9g%O+)EF|2-|Fz` z(}hrw*Nd)DaI)>t@24Z_E+C-1xzdxA%c8@BT4N~1JEqP`|`HlcxI`Q z%0x#8^wM3FZD-}HgD`6(;fTN}bC_LZI7I5js_^dG=dt5M@!K%GQYE2N#Wfl(DHG0{Giz#|G z6)8R*yPISoA@;8QgQbNw;a7a0H>21^BTSe{WvzLcm8ebI;@^arLmyBUj!qA*Rx%12 z+^|}+EWEpPo_xLgltu;U2+~NlD9ZccljMs)=q^4)`gNIBb4FL%HO=K`^96+4W621g z;R@Lc?Bkb`^h%2?BB+SGZL9V-y0*>9hwAL-Y@bA_nQ4#$zpnLEg*1-JD4%Q}ALw!6 zcYzJIWf?~kbcQ@*K%Wb41ZaLhHYX_Cy5KYwtOY^%MT?hOY=bf=tKK^jpoN1u+6Ss^MHw2soLfr*&oz~h>=P0Os^aO=*d%4xJIdfDn2B541O*Ouf=_#HyEoYkh zb5mD*%{6+n9oXSP=>+}@lZABn$&%Y5iQBY2K8^eu?ED|5QkY4lhX*!|_xs3HeBUtL zKrNJaS4e9dxR=bO*lk*>cwVE63p3+ph$`5;!;Z*|Zz4F}&C$q>JlL0k71U%_l{8v{?u10S_`j?CAiz zYWu#U{am%h*S-n*`?GqYks?Eb0-(1HH4zaL&KembB3d zCasJQSMAB^=UT&iOe#Hau8oU%l*UJqC(S=d`a?shDO-(_bz5Abx@;maT)oFtkJ+S*-GAm#xJ80RDADs!Fh4%Y&D-=dv>!K6Pbpfa zIOPgnq>rZ9)dUjT^+qC&t~OTMBlows6W316^X$w`RpBZeVgykkk48pdiWr28M`V2m z{5%E>{l%EpAF4b6bl%14QIe9&pmpWk{c5v&s@usCbA=F=((TJ~)GN!esTF(3<~1rM zcOR7Cez@!QyFoWx>g+={9Vzu&9mb2DlUb$@AfGJdyeE;emluP!$C=y92#{Nt)I9XUbX7-NYGl?HsH?C-CB_Rti>7 zs|_d`sf1>h^qsixG>BhT#YDUo!9T&qoKz~N8O#63QvMi7;z_Q;8S79JcMx6vhE-r ze_xZ|$V*Y0Zy!9PNWQbzw$?f24Vf{7Ead4n`gfec_iu|7)M%9_^qI>n(X_7a&m4`a!nP)vV2fDYN%~E3Lb*z8U{B9zSI;P(^^Cm6EZ;>d|e)=r8D?OQ+4b4P)WL$n>Q}&kIh6j zsh#I06TgQy6zi}X?fIo+CpXq5Sl(iSbpLfdK!S!b`G-0gf)CaA`FT8@i*Z)}*ks?? zSUOdh8o;rCpW=wH8f1D+dL-L=czNbZHvcD9t%BljgaM0jmXdc5NL*bzLBsTdXLqL> z%=qdk4?ZQK;~!~|tRE<_Hsw3ntCA=14{{I&HkTe6aDtFa3HyfiV8!lD%nUWbUPaNJ zA0kJLm1hkUYZAr&O)uKVGrn)&h++=)97&Rk%BMftqpswf6$M)RFxiM?$BcZUOQf*g zL!`8m6&pai&bFVYpH&+WoFFEMHjYofsZRP6Kl=wv2_3AD0RYd9k#PloY zv+Ap?Zb0UNhj$)N#cZ*G{7ucZ&~`d)%^x20U$gY*|N1bK7BBd80Ip5ctzMX>#$jK+ z(rFTZ&OJXWQ00B&FbSoSE_OxwX7u&aKoRHVev`Hl(Zscx@P~;0WX@*~$}?@3#ZCaU zQcsOz9nWPR!>eu~SY z;*q;Pc)%PS8`d-J&dX{fEGL)0U0lakf|xS$mpv>- zI5-4$vSd>$P5@}XQigLqh3Ez$qFSQTd~8OxR)mn-o!UF4csKNGLQs@9MH$i1lA}k2 z;mjc2JvM8fr=3tEXzwR!0&9h^n~MfB^Wz(*`O}o+H*ME9Z0(;#5vLS5*~(fFUB`us z-L7o5TVKA-22S;JS=G}{mcCZ=X@@DYSM29I>4G{nAc}8*1fhwUdqx%zn@pBptTw&7 z^VOclD?u8Wt|yC5;w-epa}Ou69)p4Ykp->}KW)6@-vRctB6n(gujf^i3LuCH*p59; zPS&yQ0oBZ5Em1-$4~&d}*jhNpgXm|$$v-|5lfVPNs8r@NU!CTDuN#)Re#%el(r*@4 zzAqY(D=ETN$E?*UY&?PLg{WZBP^o(WXIJX41t)=5P zVCByM=^tZ21EOPeZ+&%t1n9y<`1YkU+$QDqc*RLs<3Y~Ko5=++Pm`@x5IXdxWV$)G zw{ZeF)BFs-(QE;kqp7&ExIxIMmS3>HHrSr3cl#rTO78y#pu<`6YYGzeVt%ckuzCql zaC-C-+?~C}Rvr!v3CjZI0gGfHJyTkdy~`p25jhE6LhfEJ*G|NM1XpeM@=(T$0;^$# zj~^Wxz$#^7SD;#5pmoU^>P&G^%uwHixGsIYlH*bKi4wsdaY#t;2H_aqfzQAWn9NW7y{@P_~(OyRgb{I|XN z2{SI7DsuL#%CHO|v`xg0L%iVssdz#1IutujU&hq~u0MDGRJ8)Tu z85x>yzR86awatSlZE2+N4p9)*8;P?q4N>r*Mt=?Z=HOd3DdOvV=FX6pu^I=q#tNVc zOw@V>2h)E+jK7U*^_{T?O?11rT7z|pCXS4^TEwYLsJ-?p>o=F?@u%KK(-pfmOOT@) zuJ>m?@lXt+G+CG%TUeHlPQmjBAmU$owS`Qy@2!P<@bTkElNRC{li8JJuJ=dx%(_ri zaJfw07-fxu?<_AJUnM+B|C^6E^qvz?8$j%4+BL**1Q!42a<%C;2&=q7!JRS%`Z=@R z7FUzJ1^7D{zb0s<%97g_!$!ia4y6x>aow2rR4Q^5m`A$2y#_3yu)DMS1+>UdFq6D& zcWovD^dzJOh-)4rvU!5QZ4O)dToTAl!b2Pvk{RKP?G@`!cJVW!>b7N{oerOB z7ezE&<{b$Od!6d1$F@za>n8Jcf(}UQM?w`v4n9|8NH12~mK9N&H0IalgDmcPY_`@& zwp>VS%Y@aS3f87Vedx?dB#m$N9Ix06>3??o-=-kg6I;YA?ixSoO^R+To0}oB(G>}T z#NGNe$;rfAW=#%_@bk{tQM<-u>4_AEiooND47*s}tr>!re-cwZR3JvEvl_Q)3v_u{SlyvXut7p4~ zURHziZyIW(Zhy_vNb|`^^Pba02sq3@cXQczZ&mK%&qkU!@>?Ay;5~INPkX&IP;uup zS)kH~xmCF`Oq|oQ4<-yXsD+&O83o5+K_txj+_Q!xOfm4`pbYl*H=jS>>o2h=<5A_d z$xZ@pp7DPXz1>Z33gZvuDMV31LHi}kge8bOWEpDkoSUe=6J6c<-s)ATtccyX%_hvQ z`pSy36qDC{%7rp&#MnW8P0VmeOT=Y!SA&C=8daJKw@s~uJ?@U4XIt*5S9p*X!Ru}| zFLFA|kiHP<=^u1(CJRF3L79^lu>f!g_)zj^sm`Am6m+id!%6W%33H0Q>U5PtAFSt% z87k6iqwOLxUB)fxc_GZEY*GB)kZ7*$GZlBUN8paVu^IG2TwKb_($Rut_e?+i!KvGZ zP?-Tv-DxYf*aB_TfHq`)NH&74v7uhY)J01d#T1hnui>3_hutO9w0faBNsqKxZy42? z0eWq5+pMbXMQkmv2WP-c?`Q$h$n;ai1OXGLE0RH!Dj03&kcGKV^`a3n)3DU*-#58veCH_dJLwMUn+cPJDvObXBF_%$q zRtr%Nr12u1bIr%w*-RlStjn;V^~T1Wo7+!wMTVLWQsRmzIPHVGbJR4Jcl-aNpNn|; zgMRM85Bj;D5CoR-$KpmV^A_3=dq#1bE3{c`*KVh%>GWrXXsj&09E4Nk) zyRd0a^l;UqZrDvkW>x?IQ5J`MR5PyBBPU+X?Z5aT3gCyHJY`AGMi1qlRe7nZjN(<~ zW`W)jlQlu0d$s?<0fbot#*(I(zy-W>KJRhM^$s)7Pqk_srWFj?dVB&rJaRay>kVha zVa0W@`Yw?Dk{9-FH$8ncLyawQG(%Rs{C6Pmr$hW#c5)0w^tpNBq>zOwqVc7X!I5N3 z2tpR>zH$q;eF})PUkt?9hg&de>2EV@=KG(*$>GtXXf+g&5+4nh1tokP>*j3H@S$d%a( zBpxEEE1dJwcsWbgc-J-B1($m!F?MGLH@O_QE`a7CB^&Mr!5MH+kgMvTL{RBfw)deS zwEip}9`A=%Wu{Y^0DcNlfVouY!qhTXU{v7@TCBA?>U`Gwty6KTvNsZh-PNfmQN>Xu zZu_6?dg|$RHxXx<9+!9bth|jDAA*SsNlx1Fme##$es^HuVPt(QNmnNbs6s#Y(LM7; zVGrnczY5h`*zDE5)SxlyVF=|Z9O%yW%sfNRqtolXTf(B9*T)hAixBORlqR9E@OD|S zl%?c|<#R34y1Mn!-_`tSW;{WvXWpTT%2CI=q}Iwr))jNsv~HFf<3h63E)h9gFR#Ag z!3ysR$80^ftu`SonGgHVy1z^No>oD)Q0ze0$8M!n$nPSW8lNjY3E^^^;)V!}NcO}> z3w};Bt@p~^B4&O2q>48a$%!$-WKKGtGb|)u2B62fmbq?jF$-iCgPlopa~tL0{>?Y5 zOD-dxUD($o0pkrj%Bgxl&`ZVr)l74QB537oyd~V@UOdv9o)MLi=^ObDmi|q><2^v+K&r7=7v#_#jdJa=iQAp3-s4TC&v+yjCRICxNdJzHi6nTom ztLBwXi_}~XrVN}Yud-JJmtOvAScq*mO<{fEAP&@r2-0*b4m%AKq72Ghw}>^Uo8vt> z>wL^9{8&ul)FbvuH}CdCNK}{Wx~Z$TcqVcedaEbUo62>qnH#VFsMAK7W~!n8Iyde%=GWi?nrN(&bo0crgb zcLnZz=DM=4Pv&8T&PHdkTTzBcNVjG4KafW`2a!Zb(!z;FtAN#etBDz(>eJXVGMUi` z4d$#m>@lrGa7)*J$>_Vr)!(({or2!B;Wp85-?D7o3!F*8356BcdTZ+Wv{E5 zaZy-ZUkR@06nvi=1Os;gVa-Csjd3VKkIH5Zr~!u96+QJNg51 ze3!HtY_7lDwOhD+4*}VYSMt$Wt;rGHE*(g{uSBB%dUQ74suMK~RE;gBn$b<(CUcr) zK+Y&X)>9GK^|csb7u^<-`}o@Bmo8evNmSet!Z(WlO*jABvj_i`AMv>BZMr%Tctp}) zhpqe0-$4>?eOb|8C5M@E{t8j_jjF_qrV7W@nIn%qB%s{cAlcstH{CQuCa6g!6b9tJ zO=s&`?bVNB+jhDvFtbN_gEA;rBo{*E?yo1W`;G7RpQGi6LHWpC0lWg0@{=cz*JS}` zSVQG;^;}PPip_B@0s5yKE`TH(cddoNx#M;-I;ghg1w{k@zdB5seRF5)O-f=FIn>b6 zTh*$J)TNOMStyJCbu@G20aynAN|vUnYy zvB}4{JKxu&k0ZWLD0vct;^hb}-bESUQY0Jg?M-#2$ga(9iKaYyMZ|O>8uVR3;Q_Pf z>l)li*POiP^YGX@FyEfw5%}Hexn>mxnGa8%nV_22lO+NK^JheDq=C%qmA`-3IvHDW z-Gv0(&&lg|r|qWS9J`Uf?Wi2pF+kx@xGbzkK>w*3pFJQ#-+NDUpm3Q6-PhtZ8Jo=g zx#=rY{tbrl47)mO;UZ+mxqE-?$%tyTQ6r#5NSta3S0QbUWLT_1$yK5t6`-=i9kbyz zDq@ziz_q)Lb6R?$<1TzChpInExJ(DjQSt6{D|3f6T-xY#E$I8?eEl(iLJ{VXC5G5< zKCzx+LBNVj3~$Ii_*Zt|Uy5PNd`|d>JjRc$C+ii@L%a+yjVJ#U6aV}x$*+ycDj|*& zPlJf^(fKNToz3bw&@OaZ$3DNHtSrfLXOm(6k)kP@#srvF%Ai)xPjFGL<|ULHCyJ13 zRkVd*Npvie$rS_J7XQRl(nW6S>je3>KM^L_ycof(2Ga722h2I&t$B!$jdGY*nyeyv0DZYEc2^2uV^)Eqr zK_-Wpwy?gw`~AQVk|h5Nl`VT`Q+ zey6#2!1>$0*0wa~R^Ae*9e3=0n`0k2c3V_8UX_t?UAq3I)4#i(h8gea)8{*vbJ`d< z7DAuP{rkQA`|lgfG2ek+pe;SSm9qH*#9ZI_RL8yChAm70D}_18j4Lhiyz7j3a~7{E(Hb`4Hx{_NWyKdVjyD=C4+_*W$CKL=mEc}>hd7!Y!PJoaD%Tjxnd-Oo>T^=6d80T5m)!dY1v`O%B+m(uM!Y@p*SFy5(ZF`X;@!)XRw3I{6T3_LeXTCUs6DHoe5B zryr+AzD;if6}|}RtAMt4riJh~%kLY5I0ujiB?C_}D)XolP>Ng=e@zv<@VH)R-V=IRz`g;-A0hKHnsor~u20VQhoq0khBZxr;T1tV zN^pbPc*HxBvd_C(L+$@;t&=AgipvA7g3tP9h`5Z%l?2=0icjDW2knpacD)m_!grox zq0^dZU2ld`1r$v^LNmePvvP!~77M?#1Sc>taB=|eLD$_`Cc?WQG&1sm{oK9s`nUi0 zj&t-@?G>x=O;q>ti-H=<6_78e4rN>=iYkx@)Y^@ z5Tu z7iuWqdFo3a#{gLOLD;|B$uzbpm>R+ym&i1fKp_|zAl;Eth`)IS{FCSvX@_RgeZ0+1 zc*XQCL1==<%9?$d2I_IaZ!HdX4vH#?5CATC&$pCJK~>Qn<`KY1kC1-!sQ3cYJt?9f zdKu7BLjEfVtFz3hX4A`E=?1L$%;er(4B~6_FZi<-fIV9*-q=4B*7mutV#=Pwc_3}E zJtVBDP;Qv!Pk4MDh%wR1W8MhGSkxB6`;;Y7!C{gk~kD(<`kMMXoK{{y$g&n4= zguoJ(RI3sCi*O&}5$_kon#$|bkg>%_ym+x08u?K?<>!y-QyplPFT0a z4p|ri@%PPu7zgI<=;9KK8?r{=dCEwwN3Qzha^hP~WuCOIR9#x<0NPg#zZg;Squt0| zL>}{pR)u;Oa!92v=JW)qRD+9vb*1i-Jtw^DtN!yCZwha^$lcJq_WIV57s_CZRA-sJisfTML>g z*y3?gzdwjNX@eOtljg~9Sp9ZiVWfe>?MT$!1(QKije%f4BC65z&C|=Fwh2!#$Io3* zQYty~=xXB0se9+{;wWii>tc(ay9=eIdGhGW!b0id0yWe`_=LuJ%*5+CR~tx(o?Jm* zn~L2pGmmZDXxWg{P;^n;Zb679MCG_?RG=!6?83WFYGlkQmoYH0Po1L?dwk+&|KY?; z!=Z(^lQwzGHQ1*vps&gbvp@Et?rdQ8hY8{WiGOu7^q=3$0$1fJH=RHBA_OL8l9Q05 zpu*2b{e2u_dElyA8Ai!tFB(35%D;E{^ZbpUJ^o)0R12;m{r@LR9{-HR#a^SM5J#U5 zDzj@u$G#*R%(G|Zg`EYYR|V4QCy4NVxyZh~_cBac$w2?Yk`*iS;d9wBe6{ZlOPIwS zZgpJKH7Ibdmv8^|A_afnaP4z2{*%C!yyx2UTMbv={B$w@@iu)vT%+L-jFjjqId(*! zlf=}lQ-myfdpWm<8S3=hmnap5H;p}3_Yj}%RC&`Ze_j&a4gl?lh3MrjQsc*a(zBKB z?B6f4Gz*FgmpX3Eo$yh{hS}H7MX<$??X`@sdpWHN=WX^5eoAYvy0N_Y@;vRtM|`$` zuaot)`?#yGDT_uK*{Y@nl-ur59=BJ)KD>~=Lz1-D;^pd3Da6($uuc25;ppJXO6lIL z-F&USpXc_7D1^_3Bp@KbZpD>4fFa)Prj~**b71WH%)^Hd{d;=@aD-TWkDELUOigSi zui80Z!eMI3ip@>4dpz@fIb}ebh?HJz!&E}nmXgtDe=|1>pU;66wvzX-Hus{H=t5uKhB(TcuhI{y@pS%tDIu2Ku+d2F^ zOUmw-nwlEPWhTR-S$b`&e{XsoPk47DR5-q`7?1dYM_Q(InnKiok-)18eA~t0v@{)v zaK=VlUA9e9yo~Llbjmglq+;(v)3mx&s;waWBVWa|?V^0z>`jPp*+zI%Hv7Ak-Imgr z3!xvgFQ)8swX6yI4Nt0Psd@=m?^4w+aWYF}xRYBC%cEordn4|(?+&>+&$Bw2G{YWh z<|QJou7OzZ6K19ciPwP!9fw=4g^7Nd~kI-*&MY@@#LP+$mwG z*EEf%;P-UWyX-X=d%v)ht~Xn|Ytr@_z9zEPavYBD&cdAMf?-{;w-MHy#SmWERlI4{ zt%7F_9x1*q%O&A<$3ks&{eW{4c))n&`vGrM|5tYg%n{?g637RgU50r~ZlEU<# zYgS6;xexh;H?h~VjwVG4Lu#)(vDajmN$WGVpokU(613adlz|}$R(m_)(-G$~#9K7% zyjP$v{kY)Z>jt-C?L4(0r*+k2z1=56$=Y%Y>%agC{CUaI@xL)Z1=a7L8|Gy= zsUX&*-DD1|Kh~_Ze+yheWsX2 z_=|E@)fuF1=AeQn@UpWhu*Gq!hp!wrjx+Q=0TiXU;-TxZofcPtm3kkCf9%0j$N-LB zccVnfKaAVYFAmcLEgxInM$;ZQ=(k(g*b){tavLv?TlGF%_^14#DPGFo9!lXs!t6bgS-e(6I;8bvT%as9U`JJJ@XkS0-{6sa^r<&hVI!fa$NtaPcY_ z<{jg=aS$_1X^xEp8|`TI_=6C&H`vm`$J}i}(V#E6v5SARciF6!`@!y7QT08W>62G; zT_FxE9?s)pmrHYk8nbuC_nN&g#P1OvjP@l`L?5M{_k>ogjvIPHrBClqe9+U+7cW!_ zxGjB70FQ}5Vf7|rbv;)iVPX8@P<{{*dJiokdUq5TyxFQ22cyvX_a*5H)PVB10Y z{3t`M$>>}ra;?N90qY;PHd2okGb=2YfR}N8yDfA$K+~DqA)ANWI;F%lz@Ay2#+}uH zwM~IjMzMbBLTN7s=~-f8`EROwSvzx%l~ANh?P~d*~E2M&ZCcp zJL+#meL7OFWX^oTyTBMcbh5~GrpU=@aOQyxV(&HX>st->6hlS%6}o2HE$3gDlUw&o z#%nsmlTRYP%~+I0iwyDgY$)=fGe4xiHRRGzK6==YoN!28HV=QUl64B6ajr>@3BRoQ z#+162EEuoYW3e)^JA2Pf!P*Ryr75wp-dpwvWw!FHe`uFKBjh^Qp9xNrwj=8nfB23#1v)>}h;@iW?IDnt_ z`M~@+L3_URf#a7pd{c>^%1v5htR1SK)9~ZY{Vz-0b{(~|_4y{U zXV3lI*8Sn*n%eKL=KsMiJ$L=u=NoPt#qAfG=A zKwd|5V&@~dkl$=g_)ID(2BxpRL3esb9xTPy!}b7Gh;#nAeDqCEl@CKYbp7i4Y?^a!rc=PRpqXG=Q*^5;El>COb{Z}1%=erqICU@!D zBRuxsX}!_@N}Hg=xNz%dYt%>UT;Hszd-rXA^GZTyZFTWxKN8+iMapYpZ+ZSQaq4;% zurG^M83x%*#%Ncnt9tG)M=HKd%duwxO68jC zi_(W2gii=g${fTU+xCK6EWhiwpC&TM{G$Gx{;*o9Fzm|Fh<3L0?+$= zR^|cW%=n|LhzcFVtl`-_YpZegd&}y2w_{EqmbiVs*+Bju%=Y&c9#vA8#pBay?iv6= zpOAl>Pw;p{G7jsPropkbr8T4V&4hLibpvqz^Xcvbk|bYeHKx$4`DO9$rhoXmkS?#h-!?>Q?S zd+6q`jQ(Hjy?0oX>#{ezmIVuf1rd~HLqr5bnsgKa0Rg2-4MjQuA@ml}B`8&?(p7pF zDWN7R0#ZW_kOU$G2oRFcLLi~M51zC4S+2b|`#b-A@AdBgxWZHJXJ+o1-;{esfQ!7sv4t zQKjvj?l{50R#gu3GZ@!O?Z67amo`bKRt(6l9dgb-gJlhBDaOyj5JGv{E-{CA*S&+b z%%rD-lJYLM@j}eGH(I6D^#>N$$0$%P(E6n6e$^oltmzFGN>`5@`5Yn3*E(qD5Uh2l zVM}RX>b=ykqQ~0CQhzeJPp5w=Xl2;#Wdn3$C1&ff?OGpP1gG@|yH^Ih*3=W`;nC+=GWkdbuT68^87qi?MQ=5;3PW!e7ocA4aa9hn%FNH7a zv9{Ur*A)$;@#gxZoy^ilumgSdD5=B<$;z@s#o<#Ol*_gc5*e2lxWy$dyci3j*3=9=Gr41|OEm4rnLa6oRhORWimTA#MOD{$ z3s+oSMQbgi`G7B}reVYZ2`J)!umqHB8jqilb45S`6oGih-8TO%QhVE_BhPA-Qp2QM zPlofbb2V)Ko*5K6Z(O#c7R4hjB1|8#g@ViD6cID;hFjW{)(S@k(dS|}=Z;rcH-|ND zKF%+fvaHAOR=BqG`^}`}_tyglW#&+7)oji!r1+iHlywXnG#ZDcb^0|ES#^a)H?jT| zIQXG&yDIzf{p7}vmstabTx`2jW7`aeLn_b)JZ;4XD`R@5q!fd<2{y}9f)$43<5g>I z57JzYPjBC_sQZ+=wF===9+6tZ+WN@D*6U)b9@~C!+)35#srr1T?csT~1oyD%tV&7M;*6vyv2V;RZ7T$HD{9aLrgi!z}_^RtzBv zDzZ%<$-;j&hRgQcB4vAYdv8#Pr7jyp(+W3Kb$2=Px;?6T0a!p8{}zc7SR-8X_H@<; zL2Gg@KWo7cBvt8Oj6F1_)Jw#@%_yA*g8?E0m<>DImGQe#t7?Y6niDroOT%d{iJ30J zO>5|;d9Z~G1-1(n4fJ;%<~H6y3HTb73u62%Bhl9qx{gBvzpEwu=VMg!jSOqs!^b`YZh7ey@JFb!OaAppaJ#gpKRv|Dotw5i}C87;16 z^_Z6!Pvg}s3<3(v9t#~WI0$>r8svT5vhfB@z2%TTW-2zOi5B3mxl$vIxP&SP@$qeb zZ=c(y4u>ePj_erLkv4af3-fG+-&R+Z+|!jwLB1N!(@tAH9ELV;Sn0t`_UP;zwg`rh z2O601NT~m1iuZQ?F!!l{705D8cWDHzkGh9fduc3CK%ji8Lqz!~3G zccRccpo>l`uT{q1%C}5}a~m$YcRA_j`)p?EH#Ir@7aQX?9x(ruy=Q>%q%@@sSC3l; zQnxcpN%yyaV1qLCz_o3j>~{7)5L;uLipZ;VrmNc zb(sm1%9Z7|{vtf5 zu22e2l+Y^6(^Opb?2?;NTXwhgjQlVygjPIlYaz4=s8@jg%g!!=rvZ++aCFWjB>-wI z1za~hC~+PgLdal{cf9YX+?3pUn(AG;SZ`F4=3NTTKH85IhmADn(r+<=MKeH zb~kKy_z~q;wNf?t+BaY&u7MyF2KScQxT{T@N?qEg0Qn9u!#IDrepZf%p7Ab)4O=Rd zyR_d51e}j|=zaV7e(%ay!r%g2qrO{IZ%lKQYan+*apN57h?SR$!ak!^ey0=*pHX3s zd;aGwJwD&A80HOZ^AOdc2p0U7Q?H@WTuOj#Rx!sY5>R%l;rR2s7bUs=1g`BA2VwoslU z+5|x=$pzPx)5%+?MyT^5+|Bs?zT??+!0U9Vql1dhu~L$-H}tfwZ_-- zP#rEAHm!IoRldXptfsIff-23gCpoErEjpcnJJJMAuuPcxq{F8fnZg_MS;=I$`sFYE zQ)2dAZ=XSJS=Xvd`|Smvgo3E`W&HX2!muw@HHOVy^)>z$LW`fY;tw(EQJA_;ApE5) zxKHvVXK)(^#~!P+J#2o%X@x_B19CH70Gx704&SBUU*FC(5u3T8J0~p5p+4Vo7&WcYq*U6+{nWX%WMb>c_IT;dZ|GqT`n{ zrQJq2qcipqFF{rW{Y%Um%G?O1{X2RDGd*el)oGMb5dlY)#TD5}y+u_&W->s7Z7Y`WZq1Su#de5oy?diN~<4Cz{Z7F`O|7CD?^9_@$k9(EuF zZY~@9=6ctp))~?_zv;*3BF)@nvS(}J%zk97kL$~IC~d>q`YG=@<$SE%nK6Cu_i+t9 zk(>Se9A8UI=Z^HmiKk+h@oPJDjr*Arf-LL8Ah<*xSf(`F7w};rB^IzeTf`aZBI~sp z7M8=%C?EH7wHx9%Ta6m%~02R@MFLQO2eLCCENlBETCIvytMqNu} zzu6<*_OTB#{tK197~fG~m}>mPSctGp_;wd8CuDiHv$WIzi6DA_i^E2(%!cFo%EK^L zW*EAy&v)W+U65DS#)7{GO%%!t+LYgd*gZ&`@;Kh?vwGbx7>%|%aVoK%X3Q%0*>pFdUV9pfEk0_1a(D>#Vmbe1LKaH zL|STO6_2-B6UM+$^^hZbOSy=7muZ9x#Xx`ftW9{f%*JN=glv zwtlGvs&Vd; z1f-?DW+M?5>~f>rWejO_6ae;g(FfI^XY;h1>&KkEhb??!=LQ^FX>h34cuHds&Ugln zpZUYDT?CA~fR=&ilTkVueA2 zw574B4H*u$*!7_W<8mLez6|V8zi{2D8QO8B(|p@M+VE+ho!e1o6=dBSoh$`8bBG2UfJ~bLhiian`z8{<{Z`dxX3K9mqv*&1ninio9UsEAq);<{HE zU@^=TP&PzF8@;MqgCP{o(KeSkJi7WQuPEcod_Oa;!m#mlI5z;ni}eUJ*OvNe*z1xC z6<-S>R9AhHl5T`I01iUH{N)W(a$j{i$0lsigJ(Uu8---yY~%Gwh-==K@ql`CzW{}^ zw$NN!eo|jv&%-4!@++|^yi&NPUEQF=vz(%R;bPWeH%ezLFxyx+;F zoQPINRtH#Mm^=Zbx%?COJ-4R2G_i2i3Pf%Ad^S?(rC zm_TzG_}H4au-I%sk3!vQVp4T9pcnJ*XQD|sL>XWf$dx6WFmm;>epfZQ+yjt7ZjCkI z&y}18J)9t_8Xg^Rt#7C_|7ONO7`h&{fyKZY0FM@J7XT&CqXOr!=v&{maoN3r6=iNO zI`}i6Pq_D$5il5e!N+fMqjQ2GrP_7H7%LE_KC`?ImG&8Ee&&|uGVSgS&hB-QDU{>{ zoaIWG`qR%3PQSh7p?BT#Eg##$!pDH2h@J+2j3gIlH!*kb`l5%zib>=RIF&parCw zO#5S0_KMtmetP6^6f0*F(3yXakn{s#Am!$)A;-Gs+V@}Sb+Gap$U^~D0o)&1W%vHl z195NdPBmPo_BmLswr~u8D|ncFP+g1NDE80f`0EFTh8(#J6nEICd=t?MmY^C|*lk2` zN_*Ui;*u5#VeSLkZDgMND2cP#ewGRy)&H&DgbY|BhY&-RQX*VkY#cS@blU`)9HB_NUf?o zV$u~XZ|J;SHl59$egbJrvduEkxW?|NLQm) z7iagbbDKWlcuoD%kTd@2e%2^prd!*7puR=OxTFAO%@e+-7|U{SN;xgBY7Z%f?_Y!3 zr9b&9SYb0Aaf^nnj8NPp!N%CNwY7m@iU{CF*Otx(9Nn{M3P?v9m0t_;n0{(5R*%<_ zYwMAtoa2&m<=mwAyxwyizjG||WbaozAp1A+{8NEc&OOL9714J`z3BrQOWC-inFGC~Z&GE_^C(h#vfMi0_nmo^<|noZ0gDdZK&V ztXOtKV*H-#{w?(Gd5D$Q&VI|YzT31NwPoyv2h|W1qp!R1fROV6CZL;n3PQ+yn*?2- z2%k*QkBtOv(SeC5lu<_O?+gN;OCN+8|1Ey}ENq+}?Fa@(Pc(~!q1TF9t2%JaPyE_Jep5t zr7Gr!=mWQe)%J#bDX~dbiDK(byafwktXd zsdia=iOcW%3YGN>SvAf})CIdO&kA2n4f35unYr`)bmTnRz)l-S@UVMX5RR95vB0d% zTx7P|G*ovs2P^Co-}Y4%{CmGwerNevar+!W7gX6?oNe;nuLJMnw-7_eBp2omU_s!}k?X373zd z5uO!g@Kr)^a;fsIF+5yy3m>)WoBSSPguMrN5PQ%O_*bvS&&r4$k1_lF!JBSyf&wKv zUT0GoJqhp$NaN~X{=IiVk!Y3ss(_RD9?T>E?s=lXL+zj?GF z6oAqvJ{Zo&SVFSeVk$L3V_!ZHgmlUfM{X+1FXt=eU_$oP%QsdJ7?qnoT+Cs3w>W+= zlC#4G5T)@5N~18xiJwxL4>X2l?0WD#xBGI|!OYQ7k74aW&mlug2iR~*@Qcd6R?h1p?#OH z$_A}3IyyPM0?J|e`R=LiEjS4mdwY91v(=(wF8i1`KdC}$DaQ+`4wv{pd{C8eQg3hY z5W;+!z%!1_OmjuD_L{kwtH@p2kZd!EV?0gc+Zd659~_W*YucwR%G>e zAy+m8;9~LDh4Xu^{a6m*k15Z~N&WeSOb@J93;LGgfMCbW*~*Cd76&DWNf4=(@+~Po z|BNBLsrREbi$`zhp|%hP?+L(}cGg|H|LeyWUF$U%%9i;x|2`P?`GWhci;7P=!fhu$ zyzuC7f-n|_W8@Z2v0b_U(gWYxh6DM*k;@vPog#IzK7s98wj&>{oi=t0QIChLkFBF% z2vdWt?bB%gd<#5Fxzfe$#qjXSWJGEM*dN ztqqVcy?6Z2F4hCdPz8{b-1r*Gw^a(`%Au9Cso=JaPR8$SFW(zq!ufqmvsqgRw#Cz_ zOL)w-0#Ioy;J?wXX=fPq*SZQafK7oSTAi;A4=0&B-=P+`>oOrymnenI^QXaKILHBkx;l)0qJ?^ z+eotT_z_wjBez4SxwT#$-U`IezUe}J)Pr_#a5hR%8}`=1Yw0|1AG8NyebqlJ5*f0s zKL-c>A~8240RJm|rdH17&zsqc?QPRA{7q1-qbCNli{MtB#3z?U&Dl6Y$c=AzKD*0y z@TD{y6>rz=$N&n^fNG-U^`R!3*OUNP`N&S<+;*gk<-*~JD@TTV8ZNT>H*?`r4(jG@ zfa*8vM*!yblct7IW?;5JQk`$m;w6DLR=I`Ka->=<@}s_!oMUpsZM!*0pk!f0jw4-r z8PickCF~Yk1I6vOt*bLwFjOFq6ay6B7~2zKY)5{<^P%qre;C!PK(HB-e-@4UBk932 zbxt2w${L{O)cF{#wUlj8K7&sNkN3BOuHyMx&0-XxoK9{vHlaX2h7Dx@2ScDnUzc_pLMtK4sSMHW&E83>;p;Fx^!&+r<+Z8 zyqd#U`bI`L8xT`=P63{(4xZD5B13R!zfa67HrPFB5Ty6Z+)5=<5tBKz6hsL|Xu{$t z`^NwmL9SnE^-Cq5dWi2FgIo!cr+Q&%QREsk%u?K)2J$B})3ph6E6F2WS7Mj&t@+6I z^03wgXlo6q3ElVdSC5@DZeJNEPh2xO=TG{BGh$z7l5nJO&B(>FCVHM9xlXmBnN=?L z7!A9!vg*IRAMwhVd$DMJlK?*r?HTJ|<8P=rCm3IT;wb_dl91xsCA!&%_tuZ@ zk9nHX6|$8O#GXgh^p;yI%~>yjSob&ntS!C!oeIbK6h9E(A0_Wh_tjvQ$)8rr-UR8+ z53Rr-eJnZlXQ@2Fxw&st$ysh480Gd0S8_kU$YMN}Ut7->3s`+S;?b5d0BK}oX0;rL zt~PlwJ_#vuLA!#H4*L4~jb60Nvob$-+h(U^a_&aj2a;uVv2l4s(;8OJ*V{?|x@px7 zIPJN6{)PoKc&P)~R!(UWNNyBd^Bc8vdf?Ta%7I4{%kCj`S6|p^O-7ifRJ#nTK|~y7 zg99J|-t!wALGnrEk9aknQ1g`#m$noK6xQDI=wES9(ZVdoHdoDL)&Dj2r}^vV+X^S2)!eE%7+nH3}y zrX^li=}kjGWi{rkgoH$^R2>GVxHO0#TSl|RVU}uYYSd0)ug|>9bub z)X#6JH-9Q|87|q)kJ=vKRFwIv85S;6n}kZ%;#^f;%%0` zsveZ93NS=MeXp<|P-Fho{+BKEr~L~Ux_3zj=xm(cLSv@qG=eYe1+m}lmfEvh4_*D` zt-shx0F6`wC&o;*vfqpT`kG&iHBe-87r;;#OV07{`NE{#Qx;AVt1teJa{kF5p%FkR z(C81t>^)yN7zel`!qH*yzpnR}yB?eb@L0r0=Y)Uqp1*%}sOfIFI=8Qvei9(W;bOHvDYm~-;qZw9et@Q?=UDzDT>p0q*So^>45z=8 zvGQo^j@x(!(e*jnx@Jr&01fumcZ+?g0k;k;OlVt1=Pg^0!_9TkKac?AgX3e--Tdw0nyS5UU(-$oTB)YL}rxyKnowYbUZfrhi8VW_jGW9g#FP3AU^Xy zm4DF8#NqtQzsUyEyVm=l+h$n5mBtZo!q{X)C!vMQ($_DFQ}O0GE-pNFMe&@jJk_r% zJ-~=thS1#yxVb=S>I%Bz-@H-pV24l3EKsN^7Q$%Q7^7E^YB0G@G$cYgczY!q8iCSI zvq>w3MgGsBi1wgaJ}PX~8BZDX;fp+@ly{lv34+wT-i6;Z>==hRK9B%t) zeKNX^wAS|hWb;6`x65#~fd9Wa?1wmbjhAx^!xo#HfpwrHw`J`t-@Bh8FS%veN7&gm zzl>8#-MpV7!Jl;9;*>_p)Y8;|ZXVJC8l86ILTf{6m5^3D!?rc*)lji_>yHg!bR@Z>kp9GZp&#zLvuYTSoOQ+y-a&lvkjP7qEt@=$JiA_&+}dumkJi zhUa@5X>GuQIXlbkp8NRMTVy|A?`6*~u~5-xRCT{j|Tx~{{S03`}CvS-bicc zG+=%6YHaroqMzRlX?-QX+w+D0A=XdM-uVADu@Yq*SGs2Ivxo-ySm!OPG~K{;Ab)F3 zl$_~=ecG4kKE@4b4_pWRN~wd`nxZhn@9`3#Cepj#x=WSoX*!5WK`WUZ{``};h-&G& zY)~ZZTJB|>-r%Pp>-26R7bmz$%cn#_T6b|mf7Hs*0ksu{g}o*0Fd)*0Cf>>JeGXewU_fSjL$lmMvo$*#GZ^}I8v$T(K4Ap_ zzH5G#cpy=-RCa~bcSR#z*3S8h{Mjvz9~fve+|&x2{9tE!V+~nejt*j+nujXlx4G6( z`*7o49r10o@-Z(=C(Q2U$4m|RpLd@C9ft^n$V zc?D0+2DFlwayrs$fwQ}IZVw?D2iJX21zl43j@lYn2h8HbO=OS|9$jbHnjzVN=!R|P zS(rV>8s;=If;x$#)sk=v)eZ!*b~7)-YzkwNy@?6xbVpaKwgNvlAYqM}7G`*iVa_Ha z3i!EB75=A(G@3kK33ipvPYfoe`WxSMb&f6{^K)t33C?eOj>zCcr0gJep>XY13}4$q z)7mTlVJx?qtC&m~P%rz55A=LbSw05d-E#C z@y+>na#D8ykP|Rvfw|Mt5!V%{1NSv}AQAg*?*c@-(*+%+UY#Q#?LE!3^7-BN8umBY z_tYT;(g4&A;1b)~Gcfm`NA1GJ|Iq5^*7%Pw{6`r69!vg@F#JCv454|U_PGn+%{Ozh6o zr0Hz`JK}zP2dm(pBYZ|EX{QX2e;w-khX|h=tFN>hw?Y8cpThYh!F8<{s>cS<%g)cvpPUAabeu+f>uh*&gmaqsMZm>1fn(!N;Q(c(% z9Z!?+r{_JZtBd-tj$66AZE`3hwlR@-LMFFBy4U^d?FaARR?67=vtS5_PKnvD0;inY z*FQKZ&i&8E_4DsX6O!dYH$ZAm`}AX;yl`~#{MqE`AA*#-%BcF{$0iXHk;Yy?~7Jgwa$63Xki@T@~J zt^$CplKt^t2BM=82!*_9w`|PzBa%-&>9k#-jtIfQT&l;7Wr?JwhdKban_j38YgPD^ zApFLC^eVH|x7zdCCO6lUUQ}3J3S#b05Pt44dNt7XrF-mkJFO`*sg7qXVj}nSla9gd ztgnw8#yyj;>$EbL>fq&?#9Y!EJ++Y({mDv2;cVoEX}6=YuVMXQCzG@BSF~fETT;=t zJbTP+r@lY*pjI1yi`U&M{h{1(^*T`O@&ZhckH<>+VoQ=$J-NnO>dw!JBi22ksrs-( zzqcy9OAveIepW2dF~a@$RXZIUbNgPA?j_Pby-r)6-;EJ|;u;|4hk1ghRx;~zu>!M9d~@}%%K37VqKFs}w1 z=Vf8FLZ}@l?Qi}WND;Ta!LA>8;@mLMtfnO@?dHF8EVj+X&^M~)(iQd$JFDDQw(`0f z^$Bu$4|R3p+ipG4`hh3(y# z>T}@sLs`v-4{qN*SYfPul3g_?^5WHq4>pvh5Tm;vrR!H&-yb{$PdO5Z<%bP>rY($Z z(fs2un48j!w9z_Dpu7D5&|Wd;*%XR<9{aaT8IlShtj5W|Y;)mQ3TWY+4HhP)U z!m&=m95mkZpe>5HKQUuC7&TmK0isj6u2#DY#k+)oE~KO|f>1;Ou&%SfSOfLVHN*bt zHa^6g{LB=7VECmthDsjQmCYJf)zgdUh|7Y;9pZxvAh+Ete=vbM^B+EM^=u>APURj% zwqq!8>oYiU=f=S@3k>>hoqO_0Y4d4+US@Hfvp9)mRwq=6D(|yVhm)C~w8p)3KzF z`fxPy?M&&^q+)Ur@`U%}C1m7X7y{@B-(MV*=jZmE8e0G55#hPdqsv(2*Y~IiKezBP zJ*YQNOqYI$cA@C{h0>iPnDIco%H)lSk{Iw#80Oq9w7~x(C!C+(#F7re2BXhBT?oP>o7WC;Z2zg_2W5LrM+`s>bKy>j+v#Wsz3F zDx+-<^1`{ANm6Md7WDG!NPl_Kp!jqv7db&?baa`)hv@k7{7^8!S=nfe+gYH@cvA=$ zypxz7Y=5ZIwQB@3T`MJ^xdVTGmjLcW346{9=6Aw_r8{y8j}tW7qM5Njh$PQp9$S9l z(hq|Me6^*8qkY1U89m@-9JILW0~k~DsMuz*PXpgF|Kag)*|As^654y}?U~ZY&8;k% zb^%u;B@J7n0+iRtW0#mYf<^PwgHq+ab`0t~9YOO?o=RObz*&**Umv!X9>nc(ZDWqf zHsiQU1k7hd0|b_u@A3E)h;*ZF{BV(3iCWXE(_^pOs3Bh0S>Z&2KPoCqa}KZ;Mb``$o@@v2XT+(Iwe4Axbm|5u| zlyI!HlNYI9g_Zl(d1H|Dclj(as-Vo$ANqrhl$Kh;K?^B76D}$$LrpdLeT{Ia`8%VQdj9yav8o0RMtcc~sWyR)E+Q>w#gGR$wQ|%@so>AC>rd^eH9K zO{jXBf~gntVRD$Nj`6HZlXU>2Si_y~%UwU!?lAEn_$LdwP6E6g0Dr3^$!VRWUQ{XL zwF>zWuruM$o zLy#Zn@wr6^!T|G14aycaEJZSvm>a%GXJURhR-NoCXx`oD?^y?!FwmI82GIQ(j3B-X zSKLl(E?hm=8rpFOW?Qk@^wGarX)`u3D!}FR=SH&s#z5SQ+aL|vnJw!HUuRD2ePA$8 zO_ifZNsh5L$5?Gl)|0FflS*0Lb99bWyUrv@L{ORxxD1;U5HtTdJ+pD3SEb%kF(>Ot zgp_0NgH&k+HJ01;qprs9@2HN>OFka|G`*ww>J)!>{iAgP6NqGxttMReKT-jFCUAISUecM< znCJA+z+^MQ06Zj~EbEF><1A_3*|)*u>|pbErCOZ4t5heed3^efX*&H%GjpTa<+I2` z0e-1Ga*&XP_^SXnyl)xUqOHX)<^RDjxn%%s)$p-iwQmrv1uw3Lg?e^uC#u!9Hnzjv zb8K2VX4;^y`dQ^n!d0lbEHNc4tV00JtzcCr0fG}?(Ql<}KbQ~V&9xt23GRMfAw*bl zdq+9z_*}RmUA=*FJ$r%+U_a_lT6073E7bhb1Fn;^SlxvG-df6-7CP-KdL-3G=%Vek zY!6?9#h{s0XnC{hw4_=6p?j zmy%l{5RhT$pl4hu`~c8|$fXP3GDw`iuv(&l+qo$y2%jVzoRFjU9O^pp2#Hv%TAdq0 zP9|uw3#EVr@=j|fTAXd@4i+}8aTS)v%6C<2%Xi&%UI<=LDL+>>Zxa?o-#BDn;mf1A zvl%({bofRe@%Zs-vm!6S_pE_JXyl>gYMj=o53F()P+{C%@N?=C#~&USVa;#oP`iAaJwmJSeEK)bL zz=D?1za}vkxTNe)8ylY9?BUQjNzK+&=wsz#T0uJDE|n+v#z@^Ne^q^XknRG`DgYGd z_OpWKS6E;>6}t-!l{hi?fd+|%kK5yFaN@m8To?a#o5)GDRq)_JyYvFmliTb4Oow7! zrre&dWG<;E^)m5Q^g`aCtZ@cGS^iv%vZdfNx}+PS4{@KoTPk_S=-)rt?G$}B?dis; zbQ_qy(6k@0ZZ4N#%rni$xmYa8`Tl{p1gIyHRw+z$Zpj2p7nrdF$ar-!_DF0#{n>lLwi8XDFw~nhEVSz4W zd~L;?F07b-7bq*OuaE*VU2LrWP7otnC0!0YxDZ(wP?X=bl`?3}C%B{gY=e#ya&Wph zXq`_ipHADQNl1P)TN6*rAS|=~5WS9sj8|3KI?orpnQ5-Lc&&~R6PP>9&+%voC4VJ= zD?Lk=yQt#0UztEvbDeTyW*9N8fCT8SyB}a*>NVAzOWXr{C(?C1Jt*5Vx;sw|kJ}|? zWK0_lsGe%#@j(wCk{f12|1dvqsjE_oK#9Bp-~!&ao>d?~jyuAA#lH0My0>&EO({r- z0^i9iObSIC6_3xiDPxv0Ok)f}f|^PpNP#mGxHP(n5L&l}fPBN*!`ZZDTJ^ zN^R3jEZWHNcE*V3;P{_EZmLHX@G4G>(Z=JdjYk?)a~NDYSp|wUg=C+6@si_+W{s;WeKyp|!{3 z_E(fzjp9^>nmnn=%1FhX1b{(61AMCe;mHW+SqB9kj4Xk-PRO|Y^!&%;i8z5TL0e=m zzxTYR;+e!Jw+C@|-JW|(UOSAx>lVI(59~4F=}J+gk7K4I%M}yzXA}!VWI&qEs1onm zgWvt{O-VgsKK;j^xnvs7R>vdqlAiV*{1IrzCx&9+3e=YSE^V937QhzJbt+@akBcoA z#Q8nt%um1-N9rB&_%y^=H{hPQh`SDeXYxX}hD0SM@D zNs`S`4jy{~LVC>yawlR}mc{|T3jfzVKetbe%SiQ?I;3v{;T4t%P>CPKSseW3tD6s~ zH4AR+8(Hh$PVSuP=IUngE=?Y|T^Sh&D%zJEDz`q*h#`+>Tu@;GxsZ{5)YnA29r)H2 zVTwdtT*TW63ryC)!~TwZ4NSale7BiYD(K?Q$3-(iqsH1<5UdJ-t%JoFWuubsb+Orr zS&u@l9|moys=+KXk^P!=BE8t6yhsiuM#yn#UYhnuiqPGym6~oOOt(^&cw&2&P zsYPL-imLGKH7!Gd;{lHviHjVd5a*;=&XD3lA`ja4W;5B|o~%!s$z=`f-R$~flIS3f z{NPvKeDEQZ4M5TCh3BIH!uP~d)Uz7hLq@X0pNd-4mw1r}Ro99$*LSvHrjE}Qx~<^z zPYPPqErluEq$ieaE*b@JKu2-h<$ANruE5gBXgo}O}Ny@wwO8kaS0yos>Xkt<<8 z+A2**+}z1$S)O6wvN{Oygw za*sb3!6NXn#!r^TCb)$5@W>imMxLmSj&;6fSZtKcVHtoD5N(6Wn2b5h=s9^B^eEhI zGUnB57V_7voVlO8>O5qJP8iiwp>k)4)5I##ax7!aQv}eN&T3=aRY;a8FqAprwSc|0 zI7Zmd4u0g1HHzH5T^xN=oYt8g&$k@%>g`+6`?%-x}1JcjW7$aVCKAYO}cT#8m59E38+YF;gCCZfv4zRVa2$4YuPH6C5m z%@S*O)#X{U?i}a=_?Gr;J?EsTckKcb=E~d+p$3#Lk|6w4E~!t zqr*8yrRe1aNTqiCYe(=${auvsoJ>VBR(MGtIigajHAjY|PogKjx?Vc5e4ea{)Y5y_RFBpx5ciR~=x<-;{bb zwI6xy4EsA}XJ-#KM`kL2F@VYgx*RBiz5uTCU>_`h4RggzHl?HtBTiklEJ<)3trtYr z`zhn{CSZdTS z4oy4(Z;npuJeL|V7KFu8(7l;2(wMwQ#;a{ir{%ML_hj@lG3e84OBQJ~ldG-%v}KDs zJZsd&kTVgYq{WkB^OTd!`7>W7*OFtgPQ9#r4?Dn+$J(=yiM-of-1Vj7H({dSkL4gY zK3w&C1!ifM`HxWJh8js&`IwKaAv|``y*L?NH@#M^d%;2b;t(kK+r;`G02&XNCIsvNbfb#NS(gm3NYuS&O%^h~EmgJ>9qLc;N0KwfbpcN} zivDErz`vWUhrRYX>+@BWx%z5@c8-5H_Uf)SkYpXas&V+=KNT1%Sjw*g|97*l&N~C6 zuCmoM?)>A>x?iUU{xAZD3c3lOU4HTF@BcX5>ml3u8|Nf7Pu=v!xoqZ-H+ml@;pXKvQm%&xxm0MxuduK+=^dV( zCXxKft}~^1B{Oyy@7?TC?dnmIaNPvWVzM;#eoWBem7?^WjEjXrLq$f%iIR7^wL)qJ z{OA4L2(RN`R*fCmIC#{ld8XU6V+pJiK`&O%E*r29x4iD{Jd}7&(m4#i-|XU3WrG7B z`gC5fnx8WojVz@XnCg57{mS`Y$OR63mjkAuJJ@|s z9A@geSI!$eT~Vl;25!XgP^N!pd$OSkUi=))X7|HRCFfyt>+-@txn@B1x!)l?^A$h8 z`Ox*FXCbubo}JOYDbZxOOS*&N;LW4y0So2YnwlrKINB2u5*8{&^S-dFfK{I5pEfhQ zo1Jl#s&F%ohO6vxEJtss+b|FD*G1>Jst9W(b6*@){*@%kf0xP^7d|>uHAibbf~_09 z?l4A9lw8G@XAfs_;%hgk%yp70j=^#Wb&(c-jAc#96TC_UeyR>T?3+fkU4av z?PvV>N9&9d)253^DWA?2*V)N~)cHc*)uQ2rt>ZCoipzPfHu#X@s$Gmb$qHwIZ#>?{ zyZD=}W3Hm8aWMkRW|wg3CV2UJAN0-r=jF9VmLW?#$g;NqS6vZeSGIv4aXeezq1z3Dq1kGeL8qj@^{-*?$q%)SJA5@2*@GK$=C^Bv{9)A%Om^z!q@ z&3Wdylse?p>|l+Q%#c<$;$=NO z_qk&_*-}tdKe8PFN7vf;eB+|vG}KckjzP@^*<2Q1GP*DRm-Xy;7*bx&ENS!8_hTe%WgCByc}Ke(1umFX zM4~37nwqPb2gWCef^KJ~NMo);nh~rdx%N?qOrg=y@TnTJ*6-QzZJj7^S_^KYc!g*o z;7P!dxrfRKzVLe?J6qBknmRpAX73gFya~4iO(hKH2MZlyXd4_^C!uy385wrVwGK!( zgnxC?Ar~Cre7WLZzo{?_+4g^9P-LLW3-v6*NiaLV=jB}tn~)(mx3Ln6177nB8HpqZ zGuGO_JU6+GaV3s;hv>@GWz1WTtJY9PE{G!5p6N35%vdP0YmELCSge*kI zQd`z_alWjJ(5;nlFvfa#+fg(a&KCr0xY)jVwD7w1ZhZJZ(6Ed3R z|9K&ck!7e>PlFC!Gvqg+db*=?Qg%VgGTlrx?dCA!*aONNplynPka6j29t^BAEZSu{ zgErokVIHbwg-sndc(`$ZDWmA^4KRj!Voo6=C?{H#?0?Nd%I`EZf9|Hyei7IT_xNSA! zL#>$pVpc=Zwc1!gx?HL=`jOR_*pabJ&HyoC^IhjY-L&gxi~F#Tx!71VdE(k^ z_Wsz|0$vASoS0FJKCOsBq~NBf*mU3c%9hodY4Ut-@^@9Gy7 z(F}%|xGO-kmKtibFi~P)T@811U+beY zFU0CB3CmHYL($T~J3U;6AFZ?Kx5LZSpEFtV%lVQI3*A85|LD{!0(zXaB?L~BS)lE) zkcDt{9V2SZ_lXrrhwr@?=fg!L4Y-2SkBTI+jMxYA1)2?&5kJ%QsJ4-^-dy(@^pUT% zn7pW)v(wL38VW`0lIc}EIyIxTOlf`R+#zK5$;FNN($1+eUm$9E!ck@HjB{3Wn$%lm zgazhg!Iu9IxO#W&&DG=P=Ok`v@)|q&P{u|N2=3oiC5*HAS;7sz6g{O9|8W80xSf|X z#^>6|uxaA|VehNMqU^T5l@N&B*r5Q>rly0O3ly2!}P(+%c zyQF7e=wV=}@5bjj=RM$gU(f&FxvsgmxF_zt_F8MN{adlu-Kvb9^|y1Zx*W4a!@A^s z!N?~V61u{epy`>8yF=bj{Ln{DjPo-8{n3re+;p!v8_`0lmDU}D<@vg|eXs2@aHs^= z+`f+5c*7Sy5O;HKtRnKiIwe3~j0$EPmf)n?jX?{La_^QJtWvO6 z6`cJK=qXV%o+a&%``(R6EQez*z>X8>sR-r|XyKk!m|BB`3|+-Bcs=C!MyFm5HO50M zL3)##B1^w^%Vy{T!j9V8?A_v4oX;_ln~nR=TP%h}T2E?FW5v2h-rgDw-YZ_b1H>ZE z`&fh}O?-n@bem?!R$DY{HVxq_#xunCXr3tpiiD?Qovf(PxCDnf9oQgSOPWfhQkeZ0 z1un{DH=NH!*Uy@pTh}hG-RAHoVjWM(>XKpJ*LQykCXZrc!7W^(8!`}+nsi^*Bxe^M zywlVc&88^6nW0x0nJ{2fWYQ$%d&xwp(czZK&acSxC#>2Nuqy&#G*O{p!d0;RQ{r*s8jU$rs3 zbuun%`nI|mK5~Vz#A?mXOWk(`jqu}e(}yy(#mKXcBvE}voK*eo+b?@SYK?J*9%$(* z`#Iw{9@JCy9muZIVc>C8)MrPuOKcpE_lvJiKk2Q@2?dyQ+%B8b)CifIKOZUfO+Rj+ z-kf>mVX~-RWFkFKf>fXP`VdrWoprg3OQ6oe_t5Tx$OEThIeD?8QnOF`**NP2L@TjA zv97#dQ7GZ^cKn>ow1V6+n}P6~hVhw+7h1z?FFet?xW^4JUio2^xAU~*P#|DW+E8v& z#*n}Z!oE(HLfwKnp}xmQhm7wFLMLhnWLKS6p#HbX1`uzG;S;9z54iTaJPX@v7Z=|f z=#0&`+gHJM@|`j{+wQ7Ewh_;e;`Bs6FG#x_y+WmKZ3H0G3o>hDrpzG2aZFW5d%}!s zNUuvT_At&+-ueYS_*Nu?TG1tht3y-n2BE`Guzzi%KC2Dea#$&Qvu3@(=3)h2$w zyQKq6(vAf9NHBo2nnkOXBy8UlIcy9<+mbo)B^g&9(i6PTpWp zh~Zh@h8-WZSKW-S_e~$LZRZ3{IH`fXAx1Vw+qJ7@FCaC%=sZiCp4#ZRWYImAEi3e9 zcU?+vFKf0@vhT{sPJ_g;BoX8Rb4uLo0EssQyWUvQnMzddvX<0$#W5G#3mFIewDzs3 za1HV&gUxxKjkcmYir?{YNB z)e%5)gt!|~LQEW5;K?4#8pjKHW9VIFl|iX(A0$g? zVXd}2WMs;cmuZR)w@g9^j3p4%(m{>KZB_NO**+B=-9wHM4J*<3VpSWQyoL%A;TG6~ zfr^xe282e14z?*wzRBQiN8d$L_Ipq3h4m6q_9qn`3T&(G_dDvr+GxS58pKMvkV>UW*pZ$V-A7@>ir_(Bf z5N3pF-l%XNpF^`wZ@ar-FwK9st*a}h2Q&Ve8feJg+5HA$RZv>W2QPkZx&5W`)Al*E zvi2J3GwggUgGFR~+43!c!?Y@%!~JA2^QCFq4w68(EJLr+?YAC+MxP1fc~v0Q+6XQf z`-8e0`Fszu8k~KYKxA{ROiA&DW`Ez$=NdqLYk{&gcukJr@`{&k?qnO5I+0hSu$x%a zOW{Y>e#bz80o>CKURy7gO7vVLtv;n9!@$P9y`v+MRuX{3AR`ssWl8V;#+i4$xc$PQ zIxxL@CBQtV(p;oYfq!WiJk7s}Fr6R^W4q-2E$JJ}^7FpQTJw^kUOY}gprap5OWG5gvVzK=6MI+*T&Q=!YlH28e zS7)KMM-$|3TL0(;09y-J=qL8y)=0JB(1E>3isLfKiOSn?ntE2U2~@J6YA~+h9hN!J zH1a-Hnd00LS>nLk%ET&lk&{g0EMaaNGfi_{H4a*h`#1S9S4EeDb}Sw)ahp28w|IV* zzwo$z@u`H568cN!&+T&qI=%Z+JP3@~vaC(`3JpTE$Lc2V$%%xzQYV>)&4x1GVynIh zA3mOl_vD-6)#OsVC0YMgT4SWTIPBMa#Qdx_YL^qbh6@e%E zWWE~S86rW!S3fTDZ{r;5bgfq3Re|*E*3Jq@vP+$fwEV!+zjlkzwXplKl92}rAK|JB zcA@FGW^=2pD#vXWFIzvmn1Y8_2i@BbYUCN<8gQuDv){VTpm2eRY&2b9y)95PwB;FM z_)|&H(GgpA+UnXvv-L8^eFK4Av%@~PXhdZHp(1>Fxan$ACXA#PgQK_EeviL&WNKmZ zJr*hrxs|7ujx*)f!}JtQzi9ZW$&srIvjrsI;-DOonrow;t2OGJt03_%yD2$B%V7m% zzcsB+j%DuAW|rqrpfskEBZl%Jv0rZ=CZrE@0qE}=JJkz7@4xcx7b@NhMzuVM0yGXkYeKYfTyQ$@;hYxCx~t?WS?KVE+W*v zxBqHgMl4DNRIsffNr*Za67y~0f^l+Vw4M#GGkYuod1WAL8}C`&6(`)ceZR`F`~K|2 z`~kHE?D^6BI2&Ir zWfMy^H21|stL(oU#|hfzDEgRX*jVCDjg7WG5RUSEE7Sh10ClndXlZ}NFn#opJ7+=+ zrmoGBvpR6tq@9e{p_Z}(nhv>}cc49Sd*?f;vX7rvnLbTQXKHpRj1 zMaX4cpREa?_*pa-qK2GuOF;=12nfw?@8nP&>K9Nz>L?~=^W6-i>yQLYpY$c&4d>Q_ zCR`-b%;SW;p>$^g*CcS9uf7q~jk1OQ1P2F!U=7;5Knl^KoNA+mHy>s@u5q}}dt~dB zgmlF5Pl+In>V^uZn5TP2(=%c36SBsc01lU{3?P`Gn)dNy$9{v^xojP-94Ozs z(tWsmD}AG#y-1|VmXmr zP{sK5k9kK9(bck>TTp}AFp3fZ_f&@h36#Q7BZ`ZXI&YLdAqlS zdb)TBJhVKA+QfEX|G?d2b!4c6WJ|pt)QV2?=jJM$OfZCqT4v1vFalHLv;I?<@ylm1 z?KJJ&7w!YsTIrky4FwfUTDQ&7arss~CJQBB@snzDHZdp&U!NFwj+-@3Cd=iWn~mQb1~`bjqB5b0TuuG!llE~dUI6sbu84h@ zqpBW<*=9pJMY?n$cHl>MhV}=t4As4ld^_(H9v#4}w|>0bfh5=R^I3lQrQ+E}fxiI{ zscXz?ck^u~ZS4zF3nL7b&=)(Z;Gc$OKE@A7YH8-HX%6PYtB%v$7TRMAt#B+S+h27O z$b)@Zs-DhXK&MYgjF~xyhUFk9y^l84b?~*(HyP{`ox7iP{@izlEOO;PV(2lle=9ta zp4mQ^Qtg{~piA(oCMZ5n>VjONPn7rjWv1i!eJ&$F~-9B4U;bh zSM2JNDny3@$hqPcOZ-hiW0F%DE!*E-SpE5k_05| z7a|;iX;Eo!T21VBF`6=YGzA~Fi%fFpi6Ue9BqQI0dDI2k6K{VMk`D8~?5uQoqU=z~Bz~&P z#vY1_=MU3+NJBe$zlN=_dvguR=VT6F;aNn)eYo^Zm2O%#mal>k5y47Zu=5bFq@@YKDp4rdL4%{a+ z3<1n{@bEpp^KAE#6I-ICppnaGaPE1$gG(g0(3#(O@M`gZVouJRhTR4e>R+O2zZgT2 znkA#Q93>ZNQ}E*jY|~da235;*_Z8|Zr$@4OMrCgK9b8_59y8tXEw0wyH0i;FFfzv* zsa{ijeY=4xS;pDB6vWts z{Tk_e){FctdrJEe$V3`H$~!BKGr~FN_wgo>1l^KuIZu0LpE-Le4m{B#_3N7_p?^xN za&w6r^DP91;hs&F{Wi8SkkWm9sfhYFmYkW6i6M|p?znlE;4DqQk@6Mq)dd-1sFtq)SqJ zh5A@zni<8ZttB7J$srwp7o2)QAq4#=k-ijvf>WKDMHGM>HcwOUZpc!80u%Q!r0?x0 z`s4VtB!uUyU&=^^^PS<=Zpv;%%Upc5N38}3 zNAq;44>M|M7hm+N)DE9cojlg?lk_h(2LvBcWNP^sPv7C)_wO>NW+DFV${Wu*1Tnjn zv4&CF4Gvum9v&WgVc~WoO7^p~He94L!0meCNn^qd)+KrxS8oqSIj7{$(ZKeWbYrZmO$r@tNPAruo-8 zeoqbvvPeyBj`Z{}zkl@}pc|CjRKoZEuv*3QP6QbpaWDDI?0>r?DFTeyd&KayKMidn z4H$~ND6+k?-s4#+{odnW43bf74*6}YGencH;jk1%sBV5Z;U5VMhan&~OsH(RH6%&c zYy!(KLA*WLGn;M^tn1jvG1R({pYW`LOQ*QBzd&cq^>DJzGffxbQiW`@oo`O|3Cr%I zDVi9}CePO`5C1td&MbCZ44_t@?nUzeGP#H!!STE7`wl4Ii14C5OC2Ames@)_b4^yP z*Lsg9P)-hBWMbJlU6b*dwV~|uegU~;RhMBEM7ke&eke~hmHzI@V)JVRR(!TT%ZS2E~w3M=sw`f@{Nk(&^s|$r1$TS?$vTSS|77L>l>?aDC|fOkY&{_ zU=5~xP(IPsO0Jo!0Xo3epI14IhfFliW-fT_0YT?>btjrwfhiw;c=hv}{n_g9TA#)X z!QUpLQP)nHep&oVt^T2$)oQJ)^FlS2$zA2>M@F8>-deJI=%J=fz}+0Px6Gf8ABdR# zC1F}!fbvuZIiX*)7m)7G4>4?_1&rdo=JCWrBM$d@=bNRc;>>Rg*gZFZ9-1$8f0Eqn ztT2*ubS%D;4hhrB3S$InAY_0>w~Vl(^6yieXBX<1YtyLzo?5xN#~{dW8gw`2 zamA}?$V><-+YIE<1@?cb-hw8ZzS}`fmW7rIqp#`|&oK%9d=oiPsOuYJ9SXXq1e63( zPjKBhTpj@0!e>2oXKJD3@5qO?MRPC>>4Vge19r>_8tdpf;>j2|>|i*r@3`7vSZ}mK z3_KUsk~6?2(vGs=;U(Uk^B6SUO4BWKMtDuWWxTIjKvk9zrp==3iX2LgE z8~H$SY7c9e$I)lIz)33r{AEprxGd+6%|!7MsOCL+L3-yTz#~4??HBs7>@N%RhAWKh zh;!wyjrC;-4xRs$&8wtj-;E;YHE*cbaihS(o=ch)_1bH8{?psU&>N+Yyyv@bH zAXn|Y%4oo~Vvh4;@K7Z=e*eu?*un);GJ_iX;BBhzfh~woWkN&*rL%X+kU7<<;}YMIt;t@oAb_#?YaBO>Lju4ZW(L3r{zj zjbQC*vK9fd2Q zLpq-gSN4dosm?s6?Xz{+EwOPotjT^HPEX1=uxvY2Fgvo+;HNl}3Q5w+j^GSc@@(5? z272KKE{QFFyU)ThvXT((qJr;*@z^1JkXqsU;_H@0NoRsU%jc#m19b{cAysA{A}scR z4pAblD`PdX^4XtUT#N;VA#ml6)3}P0bXu4DxnCtpmpfv4@0U(P|JG?KM)A;|mhs^$ zC)+YD4UPlQHE0PYZyl|~$}!+Af{qF=YTqUqz$_>NC(tsK;uGcpKPlo;3{VdL6smWe zXZSHQ3Gz;-)D@*`8*Lg|9lHo5Xce)Y%_Aop7;~5?A6Ri6 zwYKzd1uL{8da(JF8UJpQZ`m8lOr9CBA+IGrs)G)Ry&qIhv0Zvp5K=4hOGOhecboDl zUx{fvDHR=O|Kh#x3bZe%RZJ35c<}5A7kXKMZ{%>Z9o#wljRqDOsVpM>BiLJN0=7kL z-Med5a**Hv+Z4>JS*ry*pDXq#xnWUNf74n*5ev<*1KTrIj6263Y_?x_VJAEug9T+d`E!LYMD-e5WHZ!0;Q@$ia&?A&@ii^Wt z`6LP^RJz8FMm~tr<+qKa-K<70D~664FVlG&?3@OwpE-c(t{AahsDT~>NfpI1!5mf7 zD*NGJiNa5*2EY?O{=!+1AT@$t3-+?UF>va%|B(AQ{JPDp;R}S6aYM&5?cllSA*1_S zvrOZ*uimk+u=wNpOG^>Z(IoLq=84e@N=~e*VtSte6@GoVPIZ^@>H&8eRWuwNvh2{U z&*r`My%s}L2npcS*-Tyf^g<8VKV6Xhkw7JW+riC+Ho#Cjyebn~J1NGaxc-C~SHILM zXX)u%YWL?sl!k64OK71r3JNW*C%%FkU4F-VV9|YXLr+sbosr!v#vd94(*B`$Ok(i$ z9MW^PTHc&OzhoKbh60LQ@|V_lc!tpjZV)p%mv##8V6VZrMPLIfN74=D#o&ehGTeLQ zzd}(B4v%uvwAC@LV=+@hPycqmH!m*jy9{P)YHJl3epaLY`pCKVBykVCU%dES-apB&`m(pm=$mb=AJSD?}vQ>_8%DxrND5R@fUQs z#&_ePt$~~j1)%T@+5AmdY4dBrC%RVL{a?%Zd!hK35(cG^t*%X{=OfOY@Q>JW0{|Ay;+ zZ^&rx$A>j_Nt_<;UvBl+7-3HUJV=^bE2w*F13ruC&*NQJ0}$Jz8^Q#^r{w(`rQ#+) zz(*->bHVB~Q2p)pj~@dF=913J9lpN=@f&IXu=^hv0LQtgEg|(ZK0n3cUsm!i;CC>Z z6*>KviT_~T|M2v`O#Dq>{;A{te`+EvL%eK~Vg9@AWnqX@kCOfM-}5>- zs#88H9!X1~ct0da8TwEi>b^oD2$YcbevzO{_5y8Bn`LWP_#IR(3A1RI=ZynxG5}BF zv5iem5C!^W6pOm<4Nj~fa>$mMctUKz(g=FS-q4qy8%p?KaCGN%5Ql`rFAIIkpn97o zZmXX(2eWuFxfXHGY9MD|9xY%P!(;)D{1F@ybBS0fvIM?D3+(J zKPwhC3Vn~lRwVt?T=T2d>SPGX&vibKP_1*^_<3kMlxJE#|64>--{Eh3@IOO;zDrU@ zo$4CLqE}t3Q)b;R183ABAeILrV)W@&#dlP;21|WrFYun6skXiL*ADwHqS+;%hV*e= zxpKxTKf}929Hi?9?@e!WFAj@}q9-7_6LwEpDq^6PpIm{%#)21PsT=5iHs;e%2=LuU z=gIrin}IRiG|;quFCcv(J=D&-_5h7pQ{&Drt}0y0=)5g8<*RY5nWv&!{%q_X%ZuJ*FilMyk7#!nhF=*Y zhcCsPgZ7~Nv!7BLsEcd_6^L)Er-sN3k%2{UFMY?NGiLpICTiST>%BdJMj}ab2@ds9 z61astP$MsNTpB&oNy6Uj`?{X_U}RsKpg!Ws!cgN@*B)P-kVj0Z{eaA*_p#DQsdZhR9_&X)g^fp^Vgk?DGhXi> z^vT{HuTD$r<;xq>kVwdx%_5A(FAH<4&e3X0WN&XJFjV?@AWKyNc+M)*$Or*wOfi95 z)uJQu9s+?Vw&=8hJ6imYhW_>6I&jxMz1JS6cYMF+_@Ak75k{S_X~-ms8qL2FMZJ9X zm)$t7Ls5X{i=Fv^-7kwc-Ik$)?hi=eBJDKLKR?x;<_y5G6N+2Sgr6>Q&>)#Ug zFVlHx3doQumEq|h*%Js%rXmV&?=*GhC(t5-~Py+|0?7e@t(f?uR@;Y#~H{gEWab9VgXt$h4b4U zM*@|TlI0Fl3nd(9AuP!{Qv*6fj6CLajP>F!DRUTgD)98x*po9>Ks*ILHQ1TJ@ORg; zwkQ^kldfgxW&SAt)np`B^4aWIu%-wOs=LvY@MfDdzPm?y7dQs zQuY4P`QY#rra{wRAUt_N}6JspA)dO^hFXXQ_9R7(sNjuhcLX9C6-X^l_BmUOlK3~X_{LiX5r zd$avro12&y9gvuNg1!T?S{2aEm*?M`uSjNKV320j%3{_lw+lDbqjA`*a74F_@A1JE zVma8%_4^Y_9h=&Gwpzcn;NK4(@0xjgN|6lVhf=Fab#qO0~^r0FGgSrVxVtwq^ce8=dQP|gW1_5T+ zGHg<_gg=J-!{s{ z6AWgl&~}E&zgkm2GpIf_3y1^OpW0Rz6ogWp(uM%oR6fDF4vB6$A9p346!nw;A2 zXeID3mo;1c`}j7$MBuB&GiLpL3;-ejQPuEIYf9^bcXAtlr>@w!IkPJC&a!z^j(G&P z`e?>`j7tWDjtihMtm^nARvH^oOX?s$fmG1x{kw-so?3pAUly{lJY z_u=8htI;CGIG%_iw$v0u&d)ME~FiKgZTTADzR zsTnJzB`Yj$Z4;eq8>LD;`N{4x+4$o=U3%#nKyCC$(zRX4hh-%Nwl!0BeFvnm)tH5_ z(n7AL5Uo-|OjLzfrziW{x;I`Ys9I4G?A;&bZ4v{TSFWnr13}AKs?nrik+PR#aF5)m zR|XE$55G)}^1)5RFI#I8&+8z|Qsg20BSoeOLlH}zJ3#mH6L1|%w8EmNLAxCx8XFI@ zuM?89u$VLk+M1ZB2L(bRwF;i-Jg^%f1CbD@^@4am{;RE|J;H;+_UXr7k46{i@whJF z5QgtZ@{e`cewRHxi^n>nGzCZ@NiEO) zhnk^jQz&PP=80(7Ra8bF0)Ce#tA_Hqs< zW7E=S1Siq5X>)|;ZC`dTd~$2_F>!)k=7_aLB?lFKlINN^ejeFqdpNv1@zKO(xJvq? zF-|QJbkBEr!&XZx7q0&N^{bk!rKt6*oh*s@_X%a$cFWx_2wb+jBGTA%*n7abJ%vVa0H#)j}7k@OXm+8|C_>~?^oN4E#|9TcCAB< zesa#oeJn2@4jgFFDX}OkV`il8%gs~}k?p*aN^(EAK1@CU7f`TlcG+9LEwI{VaD_&XpaId*6DSS443*k9(;*m8dmiQhJe8b$AlX_`}9l(5nub_ zLo14=5ggob#MgdaItR6vnF6IEnfPFMP_5GWjLGXI%EZt?ui9XIu)~MF2i}QfEPbG& zYN^qy2m}_Lt0hN5fJ1xk?28jmq;op@`;tR7Z6h(sx$|dUxA8>K0;Y9MS;3xC*UxvkWcGja>|r?cr==epKU!`VciU#>U*cOtUps?-YLPM zTly(ar&vAKey#U40Y-mjUUox3LG=)##B4Wub$hw6#_DH;hchbP6gBvRP){Y;ri{g~ zcGr1&+J}DYNAZWQ{dZ(D+Z@0+nCnDK+d%30yqS$snd#Ux>$$8wRPV9PFO z#wR+gSc#%3F4#9h)o6q4ycfUGd7IHR@*&L4rHa@4kIl9=qXRjMS;P+HMCW}>aMfF~ zL_Vw=?-;P^)UxwC48}(=C@I5^H*Wi?zf-kZnq4Pt+&Pew@n%VO^}t4Y1-zGvw*L95 zP`&Yj5{tsZ&83F5ed~>gDl$h=DY@k6mAAkH8J*#}V=NEWx0xu=4B%^`C74`TTjo*y z7aBnv)o8h@oapgSFbZ#P_icexH^uL_p}Ry?^+p{?5o|&aTk4|;PJkp3McJ5GvCS5N zuAX5_{D4P|$f2C*TUm`U0(^&eNWm}la@2oTQO}VdcFLdg4>2=1V@ge-a=Y@QK)^2@V!6P8( z3zJAitWF4*x=m5Ds^;+p-{w!6Tpm7%Go}2;oLVMRLS=hV!WC7VYiW+=Y^w47*4Y?Y z1BNz=6PacFl9_sKKc;b|X8Tdhv;LZ)bJgw}B2UfLENVR6+MD+eb9NrQ9eMEB&RBsBI8iDDhS7SZgVUuTN7T4^lT zl7oMm?f49ck@`aXk!x4>!|6yg;%pwU<=U-j_V5p2nV|m_NFuReY0i#>O;yRp)kYC?OzN8SmCU z;oE@m(F$B`MXXmA)^)g7wRlHZwO*Vk!KCeYb7{a%s|y>btAq{TnO05r`ophtGvBn1@bC zuk!nXvi?NJ^PBo@(FKoR%>GEPKNL-wWjapPLshE>Bg+CB#_2TGKd`|Zh-sjAUEOVbE2 z4^jcDh4S$|>dI~VAQsLDh^RC|n1u|R@H3QA2)n-IAW8+Su@T$$R2aqKtgCC>BNJhvsYWl?#~j=4~INHy632D0w3fPrvBieq&m^gRIF0*48RSH_MWqr#)?$D zxQ2PJ{A%}9sSkTNqH)BM@X5w^w-!S^>O-YEDe3W0ML_q-cx=Ej zsGZQf(SuthS7Pmc8MFD-)KS-B8@OI86jr|Yx4qshwZZJ?dcA{fm`p*Z^nyku3=5S_ zFDsLD*aj-AxB5=N{E-4V$c=ves;w-(D#yBxfGQ(Ir+|E~?cNo&z(I5?bLf2F(3gAs z6mN@7(q)r5k>hw?1~1yP-vV{s8|QXP3JXKn)q`I@6Lwwt@yx!*P3)PvKU&dtFi(=$ z#XeB-n?8!sgY(iJox0Ihe7@c5Pk!0PTPDc`;) zuU2Tv^cLyH#9jqawgUs$fN!j+1ivv|?&{+oMBl_F6o{D243!l61zwORMDl++J`(z9YpfHEqD@Evg$7=L~}B&b!+JLX37 zg9+_ZS^%0X7SRE0+Bq`&JI9Yqn~77~W4U@8jvD#Ay6jR~Gsi0n<7}SaO{Kgvn_ua_ zzu0ZmU;08pw{~4YK9VuX(V7xj?&F%$YGpRx_dgtd8qei$9ywShfZ8(`u$#n2e=}q7 zq?8@eO~2B==B43o^9)y7hK?AHf!#`A_V+}D4H_w@Wf0Q9} z&%)hCyV312-P2lW-@cE5cY>b*+N@E-W2OPT1#HllOUut;Per0I6A7e7qeV0{qnmjdKAx2#*(P^`BwB;IBP=i{%S#Wl+pOstA!`=}P% zihwCpI@=8!fZdB+Z&YgCH<-z5?!~BNrI>aOl4dH0!MKAjLH+1cRe$Ofd66?Xy!J<@ z(ztoaUb*SpHVBquc3NOZR_-u`&8e>nGJbp{!;jdkMYU32`tBr=TON#92wq;PK9qkG zi38Y!d&xLmL~!v4@KY22?An|D#1sn6p;YMr@@lnaebNezktx?g;A{xfX#N^899(x* z7ex@^rTaoW5rpGv*2~VcEfCx)^)@?(4x2!E$#V+5-v7fB3T&ii9i4~F*o0N7JFrx> zMHJ85Aj%Q}A)8EJvd4DmSuL6CWsST5jUR`fQVyJO`mw)%*5*}uir@gRccc}QSLpXj z8Gnuqztj#Yx7i~ysCM={+T9XhLeR@yFDX+3yfa--$`}_}2cSy^YOD@i43){;q3J&K z#Aq5~MP9v{QHhk{$Wm*{iQ@5)XEn^@GpP2G$QQ)6PfkS9K7C{}{so_$b63b59upZt zDZH>q<-|9%{TUi2~#}SqDXgj=|KMYuuJ|hqbdc45}u)*Qcoq z$(M~3OiNCa!b zXkk`Mmca=DJgjiMLnS31sNMQ7eGfk4CC`cn%|I_DuC!wWsKXMGjR(rDN{3_p5sXJL{+Ai>~!s08pVZ@7LbU z2HhU{49m{EwzdC4Elu&}ALCZ47N5o_Z+>+%OqFZP&FIx5 z!vA-A|IIrj>v$=*2;HxpJNF+^zwV2p5`;bG{XzBWKZzoUH;}$G;hUB|X}tCyP~UKT zNt_#b48P@e@oW_3-;jO|=REjaK+nsY=g*xm*l$>F`Yrni(7C#!sN^{_`NiFj4ecJA zs=aBgUS8y8#Q(VCdg;Y9#qfAX*mg0T^LX3|u3le@?-k)2a)(P5TT0;-QIVlnmbXr<2D9%tvSi9e$pq}Zgm#-nDS7RmyxE$6sY!j^LBmJT zz9Oh;id`B&aOr&3eLaP+BgHGR9jO4~eSE`=JoEaSLkb})cJtQ60gt6obndnUJOt@h z#>$^cE1boHp;9<<2RkW#XL4|d2h@+j`|uG@_ta= zW9U^s?Bn^8!>09Obg$TM@LpQ!hatoG?POlQ~5xUN~KrBE06sLQ`Q5p3BF^e9*E zQEdj1H7)kYtVya~CZ-V){Z__hSS?;BhHR5JwT&B6j#7pIfFN5jyV!YokmHERM>?VW zVg1{P42MFAC^4h+knN@OVkIuyO*!+U*fTL;$|LE4}9CT(*_* z@zW@7Nyh3Ivy7b|`NN6UG5S|5)`&PJwa2*WnEHTI=+-^G9`r7GI+0o$r7g?)nb7W| zsh!?_3Zju)JkPIAR5=Cy2okV2-Ocf8x%q3w(I&p^skp~D+N$N9zuxO%zeX8t1hm># z2XkXFday&zXlBD0q(Fg9T$Lo^TJy4N*|zX#-Pl{3K^Cy7jPj>a>t5-~uGHqxr8**x zdv`F-#&g{-oOh=h`JO*^uvAm+?SK|_@9DvuX69Z{isTg@g|*iUV6q(Os?0W{e9Rsv ze;6yWtKv8eV^pTdm%rSlYD*3jT) zj9q!2;1Vjvj>WB@)_HE}scyZ=T`DZbA^JP3a&tk(RpGOk;84-avl{vlzs9@`ZfY%gAtb|XtRf|B%d)e$@x?!FBQB1Q9XD&t zjbsizFx|W0#UK5-QGNuVvghlVL;7Gf6pUHFH%(6ACfB5^C%HfB=5(Wra)P-B*? zJ`1}YgsqO(DFM-&&fsGi4Ol@cu5nX}LNoRDUI1Z#iVrD-dr@IEHgI!IGudIra<{%s z7W#FhBy;Hj@f}&l$Nt81>mTwS`@SXGH1<%_>LxUK8(e5HB~O8#4^@my-MYgjxa|T8 zpbh#gAj zApia!juSWJ&Ru%STtjE&i zBGn_l%V}KdFq^D%Kinq*Z0~Zf-|KykS8z$FRk!AgNM{=7LR*7L6iaYYjoOB;% zp5&TlDkrNI;Xc{eTe!`zE4&*C5xe>m6L8~4Xxz88n45?pZ`hZ&3tN0<-M0eZQ;NB6 z{;u|lcFiHsN91kVpT&+uvg$3#*)-CxN&d)gPZn``rMv;0js3NqePl95=dZg6kK`1l zGd&%cEMpbJCO`tfZbsO$Ua=tg-NR7aoTMZ8Co!s7>T$ye*L^?S{nUv3_Nq|#dsS_b zTXh~=@2W#~_Tez1>NJ<*YgCvesHNfoROkg!=b_o9Xim)#$71sQ8zEH3@N_CJlp=)# zq$9v8S13EF&@O+p*TU`zO5e_QAVh%e|26jAQB5uD+jd1o6a}S#-$#@zO5T|1BqROrW#M$m6>T%96(0x#$HJM%0Y-_IAr~Z2}Z*UZ9 zf#oLSlTbl$Kdfhl1kS3o1>Dp_!F#Tw(<+}jwaR&T9L zT0Boa`}Xb9@l3K*@I$$vu3e9IuH+@fJ7j-M+~<`(cJL}qmRoIC1=t_j-EdeQd&`#q zw9Q@XjOEPRy;Wby8Xex(Y>C@u=DLRMObZJcC1+7RIp;rI(TT3~1aj=JPwO*ei3QN; zmusnm=s^OFuMRH^WXZ&NETPrGs_7F5p&2w!XBzJmSInpF;9prJU#X%kMM})gkAlVA zIxl)bx$X_`c;ooE!QQQn?rU)cZX<{9lu_&@`bOSUCtdX&h^qi|{&L3??DfzV#ZFaFcF( zVBcp&8>18s=7`|4y{n!l@k$99MYi*&0lyxO`rClcygtHz+l9t-$GNS*EMbT+uF*S; zidO|fjqkW)tJ<)XS=3&stdqtTN7BP>>AB+S6Hf^^;0i-lpBsvVdS9V%?CnaD3Ghkm zwaV04N)jo>l+&jgOjI%AzCYNC1;nD3Lj7cBqMdM+-x1gaUU=cy*1QfsKhO6 zTvXZMIxbp{T6#yQ);GyByJ(L|i<9oHPC(dUD$&O~`LoAn9VGQD*aMGnqFJJz z(ir0&$#C7~k6YeMOdy>T2yJkvCk9e~H z`0b9Eq%X_7bSCSDwLNaSj}D%=JH(Un;73wa_!$MW^I-q z_ohip5#AG1P`xiN`r(Tc^$H)vq_2@Q7k)VkRjis?ti^cO5aHa8s7{VDvm4AQTH?N6 zv3-JR+8Z%O(v|j&2&!y+e!I81ZyCCKkRPmpa2o?sVdU2)0B2g263&a~NDN22L$(QV zaJwG=w83@mgKmkYB!7+9;VUHn1cyF?-eqCuHe8eEj{0QFi|?P^N(HX$FnWX^6;s;G zgD$#?EiVTh4ML^9w)C!F%dJyTG0$dC<~MHtC=tXot=z2qEx7|i&M%;jnhiZp$J_6gM05j8&BKEe#j9=RW-fG!R-IVohX zm0Z5^&PH&jfk%0rh+n58+Gmt5C^gAAl!UV5%8fzFxMce&qTq+wyS zWa#5bQ_CG~zI9_lb;2=N-%G znm}IwCKt@oI#*V*&K8le7)CLOQaI7~CRncXc}D?R!4_R{vcQ&r|pAC`v)fPhnI z8GGp9d^d)}Ag$m5uSqF~ug~>S_gwWXDFU}%2)vF~EWt2HNOTJ|q1mhnLCm?!IUgS7 zSx1!}TqWZfHSBwTs$f1=7)v8qajS?!k(J(JqOst?0D8z+iVf8bFKUb&dF~4QECbn9 zXhOGQ7w}_F7jETyXN2J$A=Reg zeQy~6(L^Up6wKEh^RPb+j4Gw$J~9tL=Ig|Q`rK%8>zJ;uR2Wo!IsMjVGMA;b(J)u& z>1@fL%)=H7edbN4hb{EjlJo!d0`JMoLUWsTV5`OE_R7@0)6ZKJ*xuMRi98KE->Kz& zR3^>#al*zXw{w$hh*Ui(Vki)eZCE$ZQ|V$r08z;-DlnJ_D>M1T@M2%msJ73VjXp}B zP4u*mzV7Xa4P;UY{e$Jc=83v$xHwz?n}^=G+9TO}tVjAl7#`PP47Om~BhD6biX zU3o71t)$uuTQD%t38tJoY=2v9zo{0PefRLI-zs; zEMAM6@y9zR*Jako>8dYZw0sis;@E|pg#&#{yPD;UOcM!^@SilPtXq&%+rD#yt9O#e z7cbOo_Gb#+ANcM|($|TMiFI>nV9ZSB%v*w>9Z3DvUr~kV;*7?76QO>gx6?;SD-W%G zeV$ELn}H-KFDwj_FWT-QCgbx{xVaBsa75VV+}Mao=PTcJIJCI&dnI+McJ4%dLAF9) zN=}QI4aD2jZr6=bIT*BNbucklWB5U?ic`%a?c$JiIWEk}hymons1!bHSe1Y2LlP@m zOB*Pe+u>)1$9xTwGOw*)Z}PO$>i|}AqO~0!gOGjfnmOGL^N1yAtXxXcsC4-2lL+0! z&u(1BKmp)v8rP$(z>2axTeZxRsk`i&x%r~->0u22N^zn!y%~b#wVyWa#}&-JFrm!$ zzOWMk7010m?mp%)B}0lx#1mY*(7w)&+5e#C3;pE^40lX*`XS!uI{pwbeHH1bYK*kp z2mcuRy;vPb?jNe=wJKKlufOBKk$mr~h}$cJP+b?(T5*YIbMfi!MZb0yM*AIZq#$=b zD|)6B?3-_)RN9ir4(E&2q^DTSL zWjC%R#E5}MRw4rb+D>j?d&&K6?W$X9fzEtyj?viUvSc_a)Vt~BJtISb>1uu<1vaU-k3&!< zo}Xh3oSlHG+uUOZa%O=6TtJ0rB)GM4Csju?CAqeMcFB9zWx`5xpw|?thoaF4J0ITNYzH zD%*CYe1(I+^~|;<-ICi$JJ!FsUGnu=Z_9G=ee)03Vf02%2trGeUVK#o}fQs8gA=O zo{jY~N9HY%fwpXMn=`0(MEq0FfYFJLppzVmzrWVw3r&`V!ZiCLOkE zaba=RD^7<4@;TzZ?OPS>wG&AxC_Ff}xH_JcewPgnbkCXgVV2I@ud$w_?fic z{%R3}C@u?m1cvGm0))9o2XWZsg*d*`x}(04cyf(*At>q(N*uX3$+g+Ecr`j~I{sIp zJbS4CbJM4$yg@IA!mRGK=^ccu6Nhni3@T@H(&I~`!00N@XJEWJf4Z?f>i9m-H+vKT zzg;~T*Khb*?I}n1T`HE|U4C7=SUYgA?7nepG1=pq9^A*=cQ(7bQZABZHOwkn;qXrI zu|2V0#j&8=@3AV%%(PCc)sRMH#}rFJ#cfq#u^+VaK@B+Zl>SdrB1Gi1;rjL&TMt54`4Oh0>LP`5@qp9gsHh z%~|Hx5Y-T9TkQq zQ<2??eCx2=Qi6(>7@1Y)rwX;(SB6E*ZXZ7qflcJtZ7*xz7*oBf&tt=C&zFv*sqoko zXi)+%ESo6?ZG@%BUqLC7mAsXYIMb_*K+XZDNtmDB&nVqnV#=J81Eq!^Wt~15?5_%N zFLtjR+#u^*AM3nIQ76x5T;rM}=unS{rWDil3BwWX^bKJ1bsno5(8e5E>6b+wE1*&f z?L2H@#HQmON&M#cLE(Z+{&N>Vrll2(YX5wq{iF)ir&NOnU)spfzmPxp(6}Jq->tH!9A;= z+gh-z-K)HmLUWYBC^dtit$Bj(5+;ieJA4%>Mzo*n6 zGUv^a$!&P2Ntf;ByE}x_4E|5Wyh@HQ z#FvZiPVqNrUCjO3oH+d@74rN&Y?w$QJv?Ud5Xz%#l*Q!@)|WkAduoo@3JK1i7N&A-?GpIm+8r*HS>HAo(n8Ydz>iSY8Z=N?B$=7kE8CQ%iu|< zeYs-EKKq)`54-lDr1k~pY>Uu2qs*kJ*Ublrj zC?0=UtE*JAjP4QI>iVo>5){3!7DiNWtf4U&xF8af+>=gi!vw?%-o+@9Sv_NeKCUU^ zA(1Le00m=GIo~ro^4?TgA-|}?|Gkz!z4R>JytVR53!qG^5|L%vgx??+e#1Lp1HZa< zEz;Yu=VoSI5n6FRj?N%eBzG_W)8KY6eUX*P76~EdPXCysfVIEtH1%nQDLpfq$i#rA zpX|&l?uw(6Nu{j4fgZ%o=|hLc%0|UCRXPEB18D7(TLt!!QB1DxZBTOV6$RxV`ZAP? z;a&yZ3_Gb6>80n4WdY)!DsyJSZc;{8u-8xrbQ}dRO07&C`mCFKjXkEh=hW@A6x95( z-l%Z#NmU2)qpK7gznU45F1W;9L^e0?ixo*HtTTGN`etrMUsak}J;_Kc9unq)3y1u8 zU-0${jm+F4;fUQ$J*CwA#x&fN&mR)ZTBDF6<(H0ewTONpR3ip@|_~p`^yW>pTMr-zNOTPKCevkJ~ zzK-kinOS>~B)@|qg)IE;ZpV}@zT?kngVLigT!j{C5!yz0l7&EL$xiz*jCFjy5po>7 zuuq9DH=KEA-0d)t2#4|0(GuTHy!U}sc-mQaU9USMr!^Tt@u4oGn4N3~rZno*usopdN-|slmuWKAnQ&FPKRZV)M zQ6ti#p96SspnOHcxSHx!$9b8r&;6^2{AT3Q)vB_EPlA5l^lr498yvJh{18+8qj9Gb zigHC9nf?n?zjp5eNv%Qv$;g~NU3JSsCgX!jWCXktf(0T9ao)#^0!uPVF%8o|q+_zd zLcRKM_0r3_S9oVR67#~IHoMs8>GE|NEn(C_bn?!EAscPhN`#^56y=zWqG;cYzPrna zjj?8~Q|gx0jsOB<;MX>>2B#o>Y>@3l@@We$C}PnP$X0s{kCz(=@+hNRN*%{@9Gt$V z@FC_W&BAyCkKb6A(X_Z2nj;swKR5F6>k2F+bFyGtQGTGPraT_Y1VgGd&4YX}HM_Mg zk!kG`a+t3Wfq)CFbPi-6gOh)3 zC(27-$q4I-&mqPCt+)PQ`EiWgs%2_S-v4r*w`Qy=T-&@TzMZe)nmD zME~f;@$(iIw6=%$Gf&s-1sK(MydN#jZCpz6R$vXn6Z!q(+ERoe$7)F;lf;AV?`pnz zaQ~pmqguqJWN+d4A566{7fvA_)~^HdRXKX>>!)9Nt#ibm0Y1@DdH|a~bCQHweHJx~ zE>!Fn`>Lu{r{l6GM!d~BpQV%`L?<=nPV#wNSSV^z0HZl+mmG(LrfMtgB}U*3K_M;NB9f9gnZy`R)W z1o-qB7F&7mJIg`OqR?lWxiluBO{di-#>IYvSt3{Q(;dp)P+8CD%th@m{1RFZ2$1c3 z-o#MH)nTl&qKPtkKfFLaky#eUV2&+(*brO-E9_!jmIg-ZgYQeuuj|_KZ)_yq^IY%J zY#hzZkzE~syX9$0WsqUDUp(+Zq?xKhGUqk!lcP7Tywb2c>Uo6*+`n!&W(n}jpjq!S zW%uFNsK=#JRePL*974~#-y--ohS!mK+A@_TP${1hFgNucZS37fdWSJJW9~z5JD;ct zmX!UM-Rf&Kca=$0^&M$LM9AQ7b1TqN$aute{l{w~inVR|UVl% zZTy@kN8=1`)u`{4OMRsDON%_Cqb8gbblm2)`#v8)xr{CWav_prQuXfKZ)4sl;JJT! zo-BC`0+FsGcfwVwJbn~#F zJ9Eaw=m3a@5{PS~>bt*w;3+Ga~Kxt-+g054o@wl4= zeW%M#eBW_t08(6uKXcL*!4?N`$-VD@fea-oA^TN=|k95 zxv!sKGOUgBE_pHD3d?4WQoTG@HIaearm)z4Qd=rWFvh=Wly)F)L8X3%u&V`)1>{Sz zy3co|B;%IRZ!Fx8I=l``Ho1@?I>+`9##6!*^<2rcC^+=8uW)$?*$oEuZN+F*29l)b}X{`7QtPyY4whsfHRZ42z~RX+{we( z&8aS@ua1Y92U?Ka-rSJXaMJmc1=IsZ6@7J5kXZFrTt;`_{!jd`3o{fP1|Kd2eyvER zO}`VfMB2%c9=7{sqJ$$G6PJY^5j|*n`MVvaGC4WdU>ixGg_oQy%0?$NmOW}G#IGVL z=yNK!!72pkkG45SRuJMc*-DlZ(8nJO*zP-ov-lQrLZnau7W=4v*y#aN=9nY5%QuO6?` zsgJ#wtkt1mZ4BuprV;b`J8*)oXKTjgM#UeD($%o?w__4PmK*s--6CM8$pXa~&hFBi zIp}jpLs*@|csyNH`r#?PD_44@VK0!os+P3`hPthg&Tqb`G%u|0?9Oe~^R%UkUGuqq zC+G;@PF3+lyWpmEdZyJQf0C{RyvDhNDw4dqan{U(C_@WbCD z29j2B-EzqM-o<$$%i^&_$%vXsY2_fSY|1qCFb^49VkmO{$Vz%`2l1rnH5Z^9MIj zOmApH){nNp2Vj{b$!f@k|M>Ytq^k{lT18ih^9s?QB^v51Ev5Te*r&MSEs4t^1q^~C z^BgPqJM00}Yw!d04 z60v-atxSvJr?l^F*zg$ydcp`tpAs67X{sCgXs^`m>)54GY;jMWH~) zq@N?;8+tWEJBuc@H5c&hj1h4i55`q^2gEiH4A?#g*H~RNPL`ZG0NPb4jQbtN30bPz zE?RG^#-zOhkbEQi85erWb-S&MV1Ru~)Zq5y(iWZc6$AOEHQd>>SzsNeeey_=a*{V$ zNIod&4`!CsC9`)TlbOMi7=C+_ka%+Hd%15D5#s31`M8#_d8y`sYL2+&B|_!wqUDO@ zmcz_mdbpj351yUjKEZ-|B{w3cq)FI&NIvC7lwW<0SI5kjLgsXI2 z*e1r&-N}ubbnjIAeCv9Qm&0hXFd{D0%efzWj!M*bd)7zW%pQ{AJ~|&)pUByvEnBmn zk@zYzRo~=VpHV%D3ieQ$S7+p7pE~}Y+W|67-0Yv9*?U(vU%59@9O*qw8sp+0RI>-Q z+Tn`^Xc&dg!-q0SQP*di=I^O?D?}wdXph)5GZ&9}3kS)JrH|U14>pu9D6L4xpkfyX z*JT$2J+pZ)cij!4ZXJ-98rs5g1Sf_$%`87bSO`Q5q2=Q*?={aV?K%JnMg3N9`L-5p z5TOmvu5eK>oGLes0J=Bs&~O5~$$S22J_y`)UCNwHkHA87+)ua~CI!lbNLx55R8G+V&ogQIrmIn^A2D2t=_Xli=B=E`0W>>we+YXo zybUQiU4~9CW89w$&~oU69$os2BPKhqE+82>QT7K=bB+C5Ss@qa`v>GJZ)MIN=NrTl z=WyTS|C(-pC%&!S`7t(EN77cOaN|k{kuSl2u_nQ+)!x%=OS}E%+Z2S$f9^wWOI#r7 zOW31s{2`$FGjozKSpFP!Va*CCpXJ|n=Rfb?#Qx=+P4~9DbWYlJ=Gvdw=bzf9V%+}g;{W(~+mj^1j+4MPQvP=V z{w_-Y{mn;R=5yVvx)%~K|44;>?phZ;NkeWOFH-prtMDV`+EW2y%Etx7|0wPMs=a_R zNr|0d9moCp&!+p|CI9oPH0uRMszS}&H##I@y+lvig!*st@Nb~UOo1=fRQ<27y7@D| z@%yX$|4PW5Z#hvvkst;pL=U(4Zfk#j6O~UtuG=a5w^j^!bbhS}@KA@ClzV}goqO#O z!f2hstga6IacTF_2d58?_6`?L6eRqoR5^EDT! z-mbtm#GG*=#O(SfkM8_;?j!+!%WqGEfuq9udxhsUY5$oR{@f)ine*J;9BNOkPZs3Q zKl>*F`g4~G0Hls6rR+NVhaFdwOL|mmPVc-A`G+@Xv!3g=*CU&M^;b9ipNZFozcu=X zXOUV@|Igq3XS9}5l3WJLFS-A{7ys7!zq{cnjO0#X4+~b{VEXTd|Erh&zr<5Ll9K_g zeyI4vzdZy0vx%9ozkL|6a?JN`|HB)~e~)IaXE}el9Q>~l^PlFvz45z18xQYX|7Ttg zOcoJObMnQ*`xeFT6^7ft{D;LNji=l9?hRGOdkzOi_8VZ%qEkgev D?Z2m9 literal 0 HcmV?d00001 From d7e0a33a731c97cb1b63c7e7ef12674eb516486e Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 19:33:20 +0800 Subject: [PATCH 24/30] [Server-core][telemetry] update docs --- docs/en/seatunnel-engine/telemetry.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 989e8ac0b80..25c62a093bf 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -125,6 +125,8 @@ scrape_configs: # The job name assigned to scraped metrics by default. - job_name: 'seatunnel' scrape_interval: 5s + # Metrics export path + metrics_path: /hazelcast/rest/instance/metrics # List of labeled statically configured targets for this job. static_configs: # The targets specified by the static config. From ffacbfc57e13ea5887f813bc93122eb14a71951e Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 18 Jul 2023 20:56:55 +0800 Subject: [PATCH 25/30] [Server-core][Telemetry] revert example module --- docs/en/seatunnel-engine/telemetry.md | 2 +- .../seatunnel/example/engine/SeaTunnelEngineExample.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 25c62a093bf..2a96650f61d 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -142,7 +142,7 @@ For a guide on how to set up Grafana server go to the [Installation](https://gra #### Monitoring Dashboard - Add Prometheus DataSource on Grafana. -- Import `Seatunnel Cluster` monitoring dashboard by [Dashboard JSON](./grafana/grafanaDashboard.json) into Grafana. +- Import `Seatunnel Cluster` monitoring dashboard by [Dashboard JSON](./grafana/dashboard.json) into Grafana. The [effect image](../images/cluster_monitoring.png) of the dashboard diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index b43dc0cf0e3..e0bd04c0e69 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -33,7 +33,7 @@ public class SeaTunnelEngineExample { public static void main(String[] args) throws IOException, URISyntaxException, CommandException { String configurePath = - args.length > 0 ? args[0] : "/examples/stream_fakesource_to_file.conf"; + args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); @@ -41,7 +41,7 @@ public static void main(String[] args) clientCommandArgs.setJobName(Paths.get(configFile).getFileName().toString()); // Change Execution Mode to CLUSTER to use client mode, before do this, you should start // SeaTunnelEngineServerExample - clientCommandArgs.setMasterType(MasterType.CLUSTER); + clientCommandArgs.setMasterType(MasterType.LOCAL); SeaTunnel.run(clientCommandArgs.buildCommand()); } From 100007c0a66d760ad400cd99c8ec85f04c2b00d2 Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 19 Jul 2023 09:46:18 +0800 Subject: [PATCH 26/30] [Server-core][telemetry] fix code and docs --- docs/en/seatunnel-engine/telemetry.md | 78 +++++++++---------- .../engine/SeaTunnelEngineExample.java | 3 +- 2 files changed, 40 insertions(+), 41 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index 2a96650f61d..fcb16dff140 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -33,8 +33,8 @@ Note: All metrics both have the same labelName `cluster`, that's value is the co | MetricName | Type | Labels | DESCRIPTION | |-------------------------------------------|-------|------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------| -| cluster_info | Gauge | **hazelcastVersion**, hazelcastVersion
,*master**, seatunnel master address | Cluster info | -| cluster_time | Gauge | **hazelcastVersion**, hazelcastVersion
,*version**, seatunnel version | Cluster time | +| cluster_info | Gauge | **hazelcastVersion**, the version of hazelcast. **master**, seatunnel master address. | Cluster info | +| cluster_time | Gauge | **hazelcastVersion**, the version of hazelcast. | Cluster time | | node_count | Gauge | - | Cluster node total count | | node_state | Gauge | **address**, server instance address,for example: "127.0.0.1:5801" | Whether is up of seatunnel node | | hazelcast_executor_executedCount | Gauge | **type**, the type of executor, including: "async" "client" "clientBlocking" "clientQuery" "io" "offloadable" "scheduled" "system" | The hazelcast executor executedCount of seatunnel cluster node | @@ -68,43 +68,43 @@ Note: All metrics both have the same labelName `cluster`, that's value is the co ### JVM Metrics -| MetricName | Type | Labels | DESCRIPTION | -|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------| -| jvm_threads_current | Gauge | - | Current thread count of a JVM | -| jvm_threads_daemon | Gauge | - | Daemon thread count of a JVM | -| jvm_threads_peak | Gauge | - | Peak thread count of a JVM | -| jvm_threads_started_total | Counter | - | Started thread count of a JVM | -| jvm_threads_deadlocked | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers | -| jvm_threads_deadlocked_monitor | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors | -| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | Current count of threads by state | -| jvm_classes_currently_loaded | Gauge | - | The number of classes that are currently loaded in the JVM | -| jvm_classes_loaded_total | Counter | - | The total number of classes that have been loaded since the JVM has started execution | -| jvm_classes_unloaded_total | Counter | - | The total number of classes that have been unloaded since the JVM has started execution | -| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | -| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | -| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | -| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment"
**vendor**, for example: "Oracle Corporation"
**version** ,for example: "1.8.0_212-b10" | VM version info | -| process_cpu_seconds_total | Counter | - | Total user and system CPU time spent in seconds | -| process_start_time_seconds | Gauge | - | Start time of the process since unix epoch in seconds | -| process_open_fds | Gauge | - | Number of open file descriptors | -| process_max_fds | Gauge | - | Maximum number of open file descriptors | -| jvm_memory_objects_pending_finalization | Gauge | - | The number of objects waiting in the finalizer queue | -| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | Used bytes of a given JVM memory area | -| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | Committed (bytes) of a given JVM memory area | -| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | Max (bytes) of a given JVM memory area | -| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | Initial bytes of a given JVM memory area | -| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Used bytes of a given JVM memory pool | -| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Committed bytes of a given JVM memory pool | -| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Max bytes of a given JVM memory pool | -| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Initial bytes of a given JVM memory pool | -| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | -| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Used bytes after last collection of a given JVM memory pool | -| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Committed after last collection bytes of a given JVM memory pool | -| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Max bytes after last collection of a given JVM memory pool | -| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Initial after last collection bytes of a given JVM memory pool | -| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | Used bytes of a given JVM buffer pool | -| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | Bytes capacity of a given JVM buffer pool | -| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | Used buffers of a given JVM buffer pool | +| MetricName | Type | Labels | DESCRIPTION | +|--------------------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------| +| jvm_threads_current | Gauge | - | Current thread count of a JVM | +| jvm_threads_daemon | Gauge | - | Daemon thread count of a JVM | +| jvm_threads_peak | Gauge | - | Peak thread count of a JVM | +| jvm_threads_started_total | Counter | - | Started thread count of a JVM | +| jvm_threads_deadlocked | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors or ownable synchronizers | +| jvm_threads_deadlocked_monitor | Gauge | - | Cycles of JVM-threads that are in deadlock waiting to acquire object monitors | +| jvm_threads_state | Gauge | **state**, the state of jvm thread, including: "NEW" "TERMINATED" "RUNNABLE" "BLOCKED" "WAITING" "TIMED_WAITING" "UNKNOWN" | Current count of threads by state | +| jvm_classes_currently_loaded | Gauge | - | The number of classes that are currently loaded in the JVM | +| jvm_classes_loaded_total | Counter | - | The total number of classes that have been loaded since the JVM has started execution | +| jvm_classes_unloaded_total | Counter | - | The total number of classes that have been unloaded since the JVM has started execution | +| jvm_memory_pool_allocated_bytes_total | Counter | **pool**,including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_gc_collection_seconds_count | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_gc_collection_seconds_sum | Summary | **gc**,including: "PS Scavenge" "PS MarkSweep" | Time spent in a given JVM garbage collector in seconds | +| jvm_info | Gauge | **runtime**, for example: "Java(TM) SE Runtime Environment". **vendor**, for example: "Oracle Corporation". **version** ,for example: "1.8.0_212-b10" | VM version info | +| process_cpu_seconds_total | Counter | - | Total user and system CPU time spent in seconds | +| process_start_time_seconds | Gauge | - | Start time of the process since unix epoch in seconds | +| process_open_fds | Gauge | - | Number of open file descriptors | +| process_max_fds | Gauge | - | Maximum number of open file descriptors | +| jvm_memory_objects_pending_finalization | Gauge | - | The number of objects waiting in the finalizer queue | +| jvm_memory_bytes_used | Gauge | **area**, including: "heap" "noheap" | Used bytes of a given JVM memory area | +| jvm_memory_bytes_committed | Gauge | **area**, including: "heap" "noheap" | Committed (bytes) of a given JVM memory area | +| jvm_memory_bytes_max | Gauge | **area**, including:"heap" "noheap" | Max (bytes) of a given JVM memory area | +| jvm_memory_bytes_init | Gauge | **area**, including:"heap" "noheap" | Initial bytes of a given JVM memory area | +| jvm_memory_pool_bytes_used | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Used bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_committed | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Committed bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_max | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Max bytes of a given JVM memory pool | +| jvm_memory_pool_bytes_init | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Initial bytes of a given JVM memory pool | +| jvm_memory_pool_allocated_bytes_created | Gauge | **pool**, including: "Code Cache" "PS Eden Space" "PS Old Ge" "PS Survivor Space" "Compressed Class Space" "Metaspace" | Total bytes allocated in a given JVM memory pool. Only updated after GC, not continuously | +| jvm_memory_pool_collection_used_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Used bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_committed_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Committed after last collection bytes of a given JVM memory pool | +| jvm_memory_pool_collection_max_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Max bytes after last collection of a given JVM memory pool | +| jvm_memory_pool_collection_init_bytes | Gauge | **pool**, including: "PS Eden Space" "PS Old Ge" "PS Survivor Space" | Initial after last collection bytes of a given JVM memory pool | +| jvm_buffer_pool_used_bytes | Gauge | **pool**, including: "direct" "mapped" | Used bytes of a given JVM buffer pool | +| jvm_buffer_pool_capacity_bytes | Gauge | **pool**, including: "direct" "mapped" | Bytes capacity of a given JVM buffer pool | +| jvm_buffer_pool_used_buffers | Gauge | **pool**, including: "direct" "mapped" | Used buffers of a given JVM buffer pool | ### Cluster Monitoring By Prometheus & Grafana diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index e0bd04c0e69..2cb74ac48d6 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -32,8 +32,7 @@ public class SeaTunnelEngineExample { public static void main(String[] args) throws IOException, URISyntaxException, CommandException { - String configurePath = - args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; + String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); From 9323097a517abfcda59883d3b2faf927b7fa5aa7 Mon Sep 17 00:00:00 2001 From: kim-up Date: Wed, 19 Jul 2023 10:02:15 +0800 Subject: [PATCH 27/30] [Server-core][telemetry] fix telemetry.md --- docs/en/seatunnel-engine/telemetry.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/seatunnel-engine/telemetry.md b/docs/en/seatunnel-engine/telemetry.md index fcb16dff140..ad5e25cc7b1 100644 --- a/docs/en/seatunnel-engine/telemetry.md +++ b/docs/en/seatunnel-engine/telemetry.md @@ -21,9 +21,9 @@ seatunnel: ## Metrics -The [metric text of prometheus](./prometheus/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/metrics`. +The [metric text of prometheus](./prometheus/metrics.txt),which get from `http:{instanceHost}:5801/hazelcast/rest/instance/metrics`. -The [metric text of openMetrics](./openmetrics/metrics.txt),which get from `http:{instanceHost}:5701/hazelcast/rest/instance/openmetrics`. +The [metric text of openMetrics](./openmetrics/metrics.txt),which get from `http:{instanceHost}:5801/hazelcast/rest/instance/openmetrics`. Available metrics include the following categories. @@ -130,7 +130,7 @@ scrape_configs: # List of labeled statically configured targets for this job. static_configs: # The targets specified by the static config. - - targets: ['localhost:5701'] + - targets: ['localhost:5801'] # Labels assigned to all metrics scraped from the targets. # labels: [:] ``` From e724e4c27c973e0c1221315bf15dadf54169d8eb Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 25 Jul 2023 18:20:47 +0800 Subject: [PATCH 28/30] [Server-core][telemetry] revert e2e config --- .../src/test/resources/hazelcast.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml index 234493d29a6..c28e9c94d5c 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/hazelcast.yaml @@ -15,7 +15,6 @@ # limitations under the License. # - hazelcast: cluster-name: seatunnel network: From 284dd33ba08d231fecb42f0d072fbad984a5f401 Mon Sep 17 00:00:00 2001 From: kim-up Date: Thu, 27 Jul 2023 12:36:47 +0800 Subject: [PATCH 29/30] [Server-core][telemetry] fix e2e error --- .../seatunnel/engine/e2e/TelemetryApiIT.java | 409 +++++++++--------- .../telemetry/metrics/AbstractCollector.java | 8 +- .../telemetry/metrics/ExportsInstance.java | 54 --- .../metrics/ExportsInstanceInitializer.java | 28 +- .../metrics/exports/ClusterMetricExports.java | 6 +- .../metrics/exports/JobMetricExports.java | 6 +- .../exports/JobThreadPoolStatusExports.java | 6 +- .../metrics/exports/NodeMetricExports.java | 6 +- 8 files changed, 248 insertions(+), 275 deletions(-) delete mode 100644 seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java index 030d471fa5c..09f84aacae3 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/java/org/apache/seatunnel/engine/e2e/TelemetryApiIT.java @@ -110,41 +110,41 @@ public void testGetMetrics() throws InterruptedException { .body(containsString("jvm_classes")) .body(containsString("jvm_buffer_pool")) .body(containsString("process_start")) - // + // cluster_info .body(containsString("cluster_info{cluster=\"" + testClusterName)) - // + // cluster_time .body(containsString("cluster_time{cluster=\"" + testClusterName)) // Job thread pool metrics .body( - containsString( - "job_thread_pool_activeCount{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_activeCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_completedTask_total{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_completedTask_total\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_corePoolSize{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_corePoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_maximumPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_maximumPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",} 2.147483647E9")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) .body( - containsString( - "job_thread_pool_task_total{cluster=\"" + matchesRegex( + "(?s)^.*job_thread_pool_task_total\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // Job count metrics .body( containsString( @@ -204,325 +204,328 @@ public void testGetMetrics() throws InterruptedException { + "\",type=\"suspended\",} 0.0")) // Node .body( - containsString( - "node_state{cluster=\"" + matchesRegex( + "(?s)^.*node_state\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_executor_executedCount .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_executedCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_executedCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_isShutdown + .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_isShutdown{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isShutdown\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) + // hazelcast_executor_isTerminated .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_isTerminated{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_isTerminated\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_maxPoolSize .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_maxPoolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_maxPoolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_poolSize .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_poolSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_poolSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_queueRemainingCapacity .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_queueRemainingCapacity{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueRemainingCapacity\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) // hazelcast_executor_queueSize .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"async\",}")) + + "\",address=.*,type=\"async\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"client\",}")) + + "\",address=.*,type=\"client\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientBlocking\",}")) + + "\",address=.*,type=\"clientBlocking\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"clientQuery\",}")) + + "\",address=.*,type=\"clientQuery\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"io\",}")) + + "\",address=.*,type=\"io\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"offloadable\",}")) + + "\",address=.*,type=\"offloadable\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"scheduled\",}")) + + "\",address=.*,type=\"scheduled\".*$")) .body( - containsString( - "hazelcast_executor_queueSize{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_executor_queueSize\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",type=\"system\",}")) + + "\",address=.*,type=\"system\".*$")) + // hazelcast_partition_partitionCount .body( - containsString( - "hazelcast_partition_partitionCount{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_partitionCount\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_partition_activePartition .body( - containsString( - "hazelcast_partition_activePartition{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_activePartition\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_partition_isClusterSafe .body( - containsString( - "hazelcast_partition_isClusterSafe{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_isClusterSafe\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")) + + "\",address=.*$")) // hazelcast_partition_isLocalMemberSafe .body( - containsString( - "hazelcast_partition_isLocalMemberSafe{cluster=\"" + matchesRegex( + "(?s)^.*hazelcast_partition_isLocalMemberSafe\\{cluster=\"" + testClusterName - + "\",address=\"127.0.0.1:5801\",}")); + + "\",address=.*$")); } @AfterAll diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java index 99a1f15e9a1..0439fc1d77e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/AbstractCollector.java @@ -38,14 +38,14 @@ public abstract class AbstractCollector extends Collector { protected static String CLUSTER = "cluster"; protected static String ADDRESS = "address"; - protected ExportsInstance exportsInstance; + protected Node node; - public AbstractCollector(final ExportsInstance exportsInstance) { - this.exportsInstance = exportsInstance; + public AbstractCollector(final Node node) { + this.node = node; } protected Node getNode() { - return exportsInstance.getNode(); + return node; } protected ILogger getLogger(Class clazz) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java deleted file mode 100644 index 66bcdb8b64b..00000000000 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstance.java +++ /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. - */ - -package org.apache.seatunnel.engine.server.telemetry.metrics; - -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ClusterMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; -import org.apache.seatunnel.engine.server.telemetry.metrics.exports.NodeMetricExports; - -import com.hazelcast.instance.impl.Node; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.hotspot.DefaultExports; -import lombok.Getter; - -public class ExportsInstance { - - @Getter private Node node; - - public ExportsInstance(Node node) { - this.node = node; - init(); - } - - private void init() { - // initialize jvm collector - DefaultExports.initialize(); - - // register collectors - CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; - // Job info detail - new JobMetricExports(this).register(collectorRegistry); - // Thread pool status - new JobThreadPoolStatusExports(this).register(collectorRegistry); - // Node metrics - new NodeMetricExports(this).register(collectorRegistry); - // Cluster metrics - new ClusterMetricExports(this).register(collectorRegistry); - } -} diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java index ccc3f206bdd..f737fbf6c24 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/ExportsInstanceInitializer.java @@ -17,13 +17,37 @@ package org.apache.seatunnel.engine.server.telemetry.metrics; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.ClusterMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobMetricExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.JobThreadPoolStatusExports; +import org.apache.seatunnel.engine.server.telemetry.metrics.exports.NodeMetricExports; + import com.hazelcast.instance.impl.Node; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.hotspot.DefaultExports; public final class ExportsInstanceInitializer { + private static boolean initialized = false; + private ExportsInstanceInitializer() {} - public static void init(Node node) { - new ExportsInstance(node); + public static synchronized void init(Node node) { + if (!initialized) { + // initialize jvm collector + DefaultExports.initialize(); + + // register collectors + CollectorRegistry collectorRegistry = CollectorRegistry.defaultRegistry; + // Job info detail + new JobMetricExports(node).register(collectorRegistry); + // Thread pool status + new JobThreadPoolStatusExports(node).register(collectorRegistry); + // Node metrics + new NodeMetricExports(node).register(collectorRegistry); + // Cluster metrics + new ClusterMetricExports(node).register(collectorRegistry); + initialized = true; + } } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java index df5cea73fa3..51b4cb448c7 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/ClusterMetricExports.java @@ -18,9 +18,9 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import com.hazelcast.cluster.impl.MemberImpl; +import com.hazelcast.instance.impl.Node; import io.prometheus.client.GaugeMetricFamily; import java.net.UnknownHostException; @@ -30,8 +30,8 @@ public class ClusterMetricExports extends AbstractCollector { - public ClusterMetricExports(final ExportsInstance exportsInstance) { - super(exportsInstance); + public ClusterMetricExports(Node node) { + super(node); } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java index 9f24f1ee399..7ae5574fb17 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobMetricExports.java @@ -19,9 +19,9 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.JobCounter; +import com.hazelcast.instance.impl.Node; import io.prometheus.client.GaugeMetricFamily; import java.util.ArrayList; @@ -29,8 +29,8 @@ public class JobMetricExports extends AbstractCollector { - public JobMetricExports(ExportsInstance exportsInstance) { - super(exportsInstance); + public JobMetricExports(Node node) { + super(node); } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index 88b3bb0e0bd..dcd3f7f31e4 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -19,9 +19,9 @@ import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; +import com.hazelcast.instance.impl.Node; import io.prometheus.client.CounterMetricFamily; import io.prometheus.client.GaugeMetricFamily; @@ -33,8 +33,8 @@ public class JobThreadPoolStatusExports extends AbstractCollector { private static String HELP = "The %s of seatunnel coordinator job's executor cached thread pool"; - public JobThreadPoolStatusExports(ExportsInstance exportsInstance) { - super(exportsInstance); + public JobThreadPoolStatusExports(Node node) { + super(node); } @Override diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java index 25a5bfc42dd..c092ffd51b9 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/NodeMetricExports.java @@ -18,8 +18,8 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; -import org.apache.seatunnel.engine.server.telemetry.metrics.ExportsInstance; +import com.hazelcast.instance.impl.Node; import com.hazelcast.internal.jmx.InstanceMBean; import com.hazelcast.internal.jmx.PartitionServiceMBean; import io.prometheus.client.GaugeMetricFamily; @@ -29,8 +29,8 @@ public class NodeMetricExports extends AbstractCollector { - public NodeMetricExports(final ExportsInstance exportsInstance) { - super(exportsInstance); + public NodeMetricExports(Node node) { + super(node); } @Override From d307fea88f4a328f243f4d3169394be119fb1f2e Mon Sep 17 00:00:00 2001 From: kim-up Date: Tue, 8 Aug 2023 17:44:38 +0800 Subject: [PATCH 30/30] [zeta][telemetry] fix job thread metrics bug --- .../org/apache/seatunnel/engine/server/SeaTunnelServer.java | 5 +++++ .../metrics/exports/JobThreadPoolStatusExports.java | 4 +--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java index f75f5af8e4d..2486e2ba496 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/SeaTunnelServer.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.engine.server.execution.TaskGroupLocation; import org.apache.seatunnel.engine.server.service.slot.DefaultSlotService; import org.apache.seatunnel.engine.server.service.slot.SlotService; +import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; import com.hazelcast.internal.services.ManagedService; import com.hazelcast.internal.services.MembershipAwareService; @@ -253,4 +254,8 @@ private void printExecutionInfo() { coordinatorService.printJobDetailInfo(); } } + + public ThreadPoolStatus getThreadPoolStatusMetrics() { + return coordinatorService.getThreadPoolStatusMetrics(); + } } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java index dcd3f7f31e4..80ee89343e0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/telemetry/metrics/exports/JobThreadPoolStatusExports.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.engine.server.telemetry.metrics.exports; -import org.apache.seatunnel.engine.server.CoordinatorService; import org.apache.seatunnel.engine.server.telemetry.metrics.AbstractCollector; import org.apache.seatunnel.engine.server.telemetry.metrics.entity.ThreadPoolStatus; @@ -40,11 +39,10 @@ public JobThreadPoolStatusExports(Node node) { @Override public List collect() { List mfs = new ArrayList(); - CoordinatorService coordinatorService = getCoordinatorService(); List labelValues = labelValues(localAddress()); - ThreadPoolStatus threadPoolStatusMetrics = coordinatorService.getThreadPoolStatusMetrics(); + ThreadPoolStatus threadPoolStatusMetrics = getServer().getThreadPoolStatusMetrics(); List labelNames = clusterLabelNames(ADDRESS); GaugeMetricFamily activeCount =