From 2504f51da76f256298a50ee27ee09341a3594670 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Wed, 3 Dec 2025 13:40:53 -0600 Subject: [PATCH 01/29] Added iceberg metrics to be exposed in /metrics REST API call. --- .../com/altinity/ice/rest/catalog/Main.java | 43 +- .../metrics/PrometheusMetricsReporter.java | 546 ++++++++++++++++++ .../internal/rest/RESTCatalogAdapter.java | 18 +- 3 files changed, 595 insertions(+), 12 deletions(-) create mode 100644 ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/Main.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/Main.java index c842705..fc07a28 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/Main.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/Main.java @@ -27,6 +27,7 @@ import com.altinity.ice.rest.catalog.internal.maintenance.ManifestCompaction; import com.altinity.ice.rest.catalog.internal.maintenance.OrphanCleanup; import com.altinity.ice.rest.catalog.internal.maintenance.SnapshotCleanup; +import com.altinity.ice.rest.catalog.internal.metrics.PrometheusMetricsReporter; import com.altinity.ice.rest.catalog.internal.rest.RESTCatalogAdapter; import com.altinity.ice.rest.catalog.internal.rest.RESTCatalogAuthorizationHandler; import com.altinity.ice.rest.catalog.internal.rest.RESTCatalogHandler; @@ -202,8 +203,13 @@ void performMaintenance( } private static Server createServer( - String host, int port, Catalog catalog, Config config, Map icebergConfig) { - var s = createBaseServer(catalog, config, icebergConfig, true); + String host, + int port, + Catalog catalog, + Config config, + Map icebergConfig, + PrometheusMetricsReporter metricsReporter) { + var s = createBaseServer(catalog, config, icebergConfig, true, metricsReporter); ServerConnector connector = new ServerConnector(s); connector.setHost(host); connector.setPort(port); @@ -212,8 +218,13 @@ private static Server createServer( } private static Server createAdminServer( - String host, int port, Catalog catalog, Config config, Map icebergConfig) { - var s = createBaseServer(catalog, config, icebergConfig, false); + String host, + int port, + Catalog catalog, + Config config, + Map icebergConfig, + PrometheusMetricsReporter metricsReporter) { + var s = createBaseServer(catalog, config, icebergConfig, false, metricsReporter); ServerConnector connector = new ServerConnector(s); connector.setHost(host); connector.setPort(port); @@ -222,7 +233,11 @@ private static Server createAdminServer( } private static Server createBaseServer( - Catalog catalog, Config config, Map icebergConfig, boolean requireAuth) { + Catalog catalog, + Config config, + Map icebergConfig, + boolean requireAuth, + PrometheusMetricsReporter metricsReporter) { var mux = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); mux.insertHandler(new GzipHandler()); // TODO: RequestLogHandler @@ -237,7 +252,7 @@ private static Server createBaseServer( if (requireAuth) { mux.insertHandler(createAuthorizationHandler(config.bearerTokens(), config)); - restCatalogAdapter = new RESTCatalogAdapter(catalog); + restCatalogAdapter = new RESTCatalogAdapter(catalog, metricsReporter); var globalConfig = config.toIcebergConfigDefaults(); if (!globalConfig.isEmpty()) { restCatalogAdapter = new RESTCatalogMiddlewareConfig(restCatalogAdapter, globalConfig); @@ -257,7 +272,7 @@ private static Server createBaseServer( new RESTCatalogMiddlewareCredentials(restCatalogAdapter, auth), auth); } } else { - restCatalogAdapter = new RESTCatalogAdapter(catalog); + restCatalogAdapter = new RESTCatalogAdapter(catalog, metricsReporter); var globalConfig = config.toIcebergConfigDefaults(); if (!globalConfig.isEmpty()) { restCatalogAdapter = new RESTCatalogMiddlewareConfig(restCatalogAdapter, globalConfig); @@ -401,6 +416,9 @@ public Integer call() throws Exception { logger.info("Catalog maintenance disabled (no maintenance schedule specified)"); } + // Initialize Iceberg metrics reporter for Prometheus (singleton) + PrometheusMetricsReporter metricsReporter = PrometheusMetricsReporter.getInstance(); + // TODO: ensure all http handlers are hooked in JvmMetrics.builder().register(); @@ -414,14 +432,21 @@ public Integer call() throws Exception { adminHostAndPort.getPort(), catalog, config, - icebergConfig); + icebergConfig, + metricsReporter); adminServer.start(); logger.warn("Serving admin endpoint at http://{}/v1/{config,*}", adminHostAndPort); } HostAndPort hostAndPort = HostAndPort.fromString(config.addr()); Server httpServer = - createServer(hostAndPort.getHost(), hostAndPort.getPort(), catalog, config, icebergConfig); + createServer( + hostAndPort.getHost(), + hostAndPort.getPort(), + catalog, + config, + icebergConfig, + metricsReporter); httpServer.start(); logger.info("Serving http://{}/v1/{config,*}", hostAndPort); diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java new file mode 100644 index 0000000..557623e --- /dev/null +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java @@ -0,0 +1,546 @@ +/* + * Copyright (c) 2025 Altinity Inc and/or its affiliates. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.altinity.ice.rest.catalog.internal.metrics; + +import io.prometheus.metrics.core.datapoints.DistributionDataPoint; +import io.prometheus.metrics.core.metrics.Counter; +import io.prometheus.metrics.core.metrics.Histogram; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.metrics.CommitReport; +import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.ScanReport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Prometheus-based metrics reporter for Iceberg operations. This reporter exposes Iceberg scan + * and commit metrics via Prometheus, making them available at the /metrics endpoint. + * + *

This class uses a singleton pattern because Prometheus metrics can only be registered once per + * JVM. + * + *

All duration metrics use Histograms instead of Summaries to allow aggregation across multiple + * instances in distributed deployments. + */ +public class PrometheusMetricsReporter { + + private static final Logger logger = LoggerFactory.getLogger(PrometheusMetricsReporter.class); + + // Singleton instance + private static volatile PrometheusMetricsReporter instance; + private static final Object lock = new Object(); + + // Common label names + private static final String[] SCAN_LABELS = {"catalog", "namespace", "table"}; + private static final String[] COMMIT_LABELS = {"catalog", "namespace", "table", "operation"}; + + // Duration histogram buckets (in seconds) - suitable for typical Iceberg operations + private static final double[] DURATION_BUCKETS = { + 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, 2.5, 5, 10, 30, 60 + }; + + // Scan metrics counters + private final Counter scansTotal; + private final Counter scanResultDataFiles; + private final Counter scanResultDeleteFiles; + private final Counter scanIndexedDeleteFiles; + private final Counter scanPositionalDeleteFiles; + private final Counter scanEqualityDeleteFiles; + private final Counter scanTotalDataManifests; + private final Counter scanTotalDeleteManifests; + private final Counter scanScannedDataManifests; + private final Counter scanSkippedDataManifests; + private final Counter scanTotalFileSizeBytes; + private final Counter scanTotalDeleteFileSizeBytes; + + // Scan timing metrics (Histogram for aggregation across instances) + private final Histogram scanPlanningDuration; + private final Histogram scanDataFilesPerScan; + + // Commit metrics counters + private final Counter commitsTotal; + private final Counter commitAddedDataFiles; + private final Counter commitRemovedDataFiles; + private final Counter commitAddedDeleteFiles; + private final Counter commitRemovedDeleteFiles; + private final Counter commitAddedRecords; + private final Counter commitRemovedRecords; + private final Counter commitAddedEqualityDeletes; + private final Counter commitTotalFilesSizeBytes; + + // Commit timing metrics (Histogram for aggregation across instances) + private final Histogram commitDuration; + + // Error counter + private final Counter metricsReportErrors; + + // Info metric to confirm reporter is active + private final Counter metricsReporterInfo; + + /** Returns the singleton instance of the metrics reporter. */ + public static PrometheusMetricsReporter getInstance() { + if (instance == null) { + synchronized (lock) { + if (instance == null) { + instance = new PrometheusMetricsReporter(); + } + } + } + return instance; + } + + private PrometheusMetricsReporter() { + // Info metric - incremented once to confirm reporter is active and visible in /metrics + this.metricsReporterInfo = + Counter.builder() + .name("iceberg_metrics_reporter_info") + .help("Iceberg metrics reporter info (value 1 means reporter is active)") + .register(); + this.metricsReporterInfo.inc(); // Make it visible immediately + + // Error counter + this.metricsReportErrors = + Counter.builder() + .name("iceberg_metrics_report_errors_total") + .help("Total number of errors while processing metrics reports") + .labelNames("type") + .register(); + + // Scan metrics + this.scansTotal = + Counter.builder() + .name("iceberg_scans_total") + .help("Total number of Iceberg table scans") + .labelNames(SCAN_LABELS) + .register(); + + this.scanResultDataFiles = + Counter.builder() + .name("iceberg_scan_result_data_files_total") + .help("Total number of data files in scan results") + .labelNames(SCAN_LABELS) + .register(); + + this.scanResultDeleteFiles = + Counter.builder() + .name("iceberg_scan_result_delete_files_total") + .help("Total number of delete files in scan results") + .labelNames(SCAN_LABELS) + .register(); + + this.scanIndexedDeleteFiles = + Counter.builder() + .name("iceberg_scan_indexed_delete_files_total") + .help("Total number of indexed delete files in scan results") + .labelNames(SCAN_LABELS) + .register(); + + this.scanPositionalDeleteFiles = + Counter.builder() + .name("iceberg_scan_positional_delete_files_total") + .help("Total number of positional delete files in scan results") + .labelNames(SCAN_LABELS) + .register(); + + this.scanEqualityDeleteFiles = + Counter.builder() + .name("iceberg_scan_equality_delete_files_total") + .help("Total number of equality delete files in scan results") + .labelNames(SCAN_LABELS) + .register(); + + this.scanTotalDataManifests = + Counter.builder() + .name("iceberg_scan_total_data_manifests") + .help("Total number of data manifests considered during scans") + .labelNames(SCAN_LABELS) + .register(); + + this.scanTotalDeleteManifests = + Counter.builder() + .name("iceberg_scan_total_delete_manifests") + .help("Total number of delete manifests considered during scans") + .labelNames(SCAN_LABELS) + .register(); + + this.scanScannedDataManifests = + Counter.builder() + .name("iceberg_scan_scanned_data_manifests") + .help("Total number of data manifests actually scanned") + .labelNames(SCAN_LABELS) + .register(); + + this.scanSkippedDataManifests = + Counter.builder() + .name("iceberg_scan_skipped_data_manifests") + .help("Total number of data manifests skipped during scans") + .labelNames(SCAN_LABELS) + .register(); + + this.scanTotalFileSizeBytes = + Counter.builder() + .name("iceberg_scan_total_file_size_bytes") + .help("Total file size in bytes for scanned data files") + .labelNames(SCAN_LABELS) + .register(); + + this.scanTotalDeleteFileSizeBytes = + Counter.builder() + .name("iceberg_scan_total_delete_file_size_bytes") + .help("Total file size in bytes for scanned delete files") + .labelNames(SCAN_LABELS) + .register(); + + // Scan timing - using Histogram for aggregation across instances + this.scanPlanningDuration = + Histogram.builder() + .name("iceberg_scan_planning_duration_seconds") + .help("Duration of scan planning in seconds") + .labelNames(SCAN_LABELS) + .classicUpperBounds(DURATION_BUCKETS) + .register(); + + this.scanDataFilesPerScan = + Histogram.builder() + .name("iceberg_scan_data_files_per_scan") + .help("Distribution of data files per scan") + .labelNames(SCAN_LABELS) + .classicExponentialUpperBounds(1, 2, 10) + .register(); + + // Commit metrics + this.commitsTotal = + Counter.builder() + .name("iceberg_commits_total") + .help("Total number of Iceberg table commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitAddedDataFiles = + Counter.builder() + .name("iceberg_commit_added_data_files_total") + .help("Total number of data files added in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitRemovedDataFiles = + Counter.builder() + .name("iceberg_commit_removed_data_files_total") + .help("Total number of data files removed in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitAddedDeleteFiles = + Counter.builder() + .name("iceberg_commit_added_delete_files_total") + .help("Total number of delete files added in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitRemovedDeleteFiles = + Counter.builder() + .name("iceberg_commit_removed_delete_files_total") + .help("Total number of delete files removed in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitAddedRecords = + Counter.builder() + .name("iceberg_commit_added_records_total") + .help("Total number of records added in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitRemovedRecords = + Counter.builder() + .name("iceberg_commit_removed_records_total") + .help("Total number of records removed in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitAddedEqualityDeletes = + Counter.builder() + .name("iceberg_commit_added_equality_deletes_total") + .help("Total number of equality deletes added in commits") + .labelNames(COMMIT_LABELS) + .register(); + + this.commitTotalFilesSizeBytes = + Counter.builder() + .name("iceberg_commit_total_files_size_bytes") + .help("Total size in bytes of files involved in commits") + .labelNames(COMMIT_LABELS) + .register(); + + // Commit timing - using Histogram for aggregation across instances + this.commitDuration = + Histogram.builder() + .name("iceberg_commit_duration_seconds") + .help("Duration of commit operations in seconds") + .labelNames(COMMIT_LABELS) + .classicUpperBounds(DURATION_BUCKETS) + .register(); + + logger.info("Prometheus Iceberg metrics reporter initialized"); + } + + /** + * Report metrics from an Iceberg MetricsReport. + * + * @param catalogName the catalog name for multi-catalog deployments + * @param report the metrics report to process + */ + public void report(String catalogName, MetricsReport report) { + if (report == null) { + return; + } + + String catalog = catalogName != null ? catalogName : "default"; + + try { + if (report instanceof ScanReport) { + reportScanMetrics(catalog, (ScanReport) report); + } else if (report instanceof CommitReport) { + reportCommitMetrics(catalog, (CommitReport) report); + } else { + logger.debug("Unknown metrics report type: {}", report.getClass().getName()); + } + } catch (Exception e) { + String reportType = report.getClass().getSimpleName(); + logger.warn("Error processing {} metrics report: {}", reportType, e.getMessage()); + metricsReportErrors.labelValues(reportType).inc(); + } + } + + /** + * Report metrics from an Iceberg MetricsReport using default catalog name. + * + * @param report the metrics report to process + */ + public void report(MetricsReport report) { + report(null, report); + } + + private void reportScanMetrics(String catalog, ScanReport report) { + String fullTableName = report.tableName(); + String namespace = extractNamespace(fullTableName); + String table = extractTableName(fullTableName); + + logger.debug( + "Recording scan metrics for catalog: {}, namespace: {}, table: {}", + catalog, + namespace, + table); + + scansTotal.labelValues(catalog, namespace, table).inc(); + + var metrics = report.scanMetrics(); + if (metrics == null) { + return; + } + + // File counts + if (metrics.resultDataFiles() != null) { + long count = metrics.resultDataFiles().value(); + scanResultDataFiles.labelValues(catalog, namespace, table).inc(count); + scanDataFilesPerScan.labelValues(catalog, namespace, table).observe(count); + } + + if (metrics.resultDeleteFiles() != null) { + scanResultDeleteFiles + .labelValues(catalog, namespace, table) + .inc(metrics.resultDeleteFiles().value()); + } + + // Additional delete file metrics (if available in the Iceberg version) + if (metrics.indexedDeleteFiles() != null) { + scanIndexedDeleteFiles + .labelValues(catalog, namespace, table) + .inc(metrics.indexedDeleteFiles().value()); + } + + if (metrics.positionalDeleteFiles() != null) { + scanPositionalDeleteFiles + .labelValues(catalog, namespace, table) + .inc(metrics.positionalDeleteFiles().value()); + } + + if (metrics.equalityDeleteFiles() != null) { + scanEqualityDeleteFiles + .labelValues(catalog, namespace, table) + .inc(metrics.equalityDeleteFiles().value()); + } + + // Manifest counts + if (metrics.totalDataManifests() != null) { + scanTotalDataManifests + .labelValues(catalog, namespace, table) + .inc(metrics.totalDataManifests().value()); + } + + if (metrics.totalDeleteManifests() != null) { + scanTotalDeleteManifests + .labelValues(catalog, namespace, table) + .inc(metrics.totalDeleteManifests().value()); + } + + if (metrics.scannedDataManifests() != null) { + scanScannedDataManifests + .labelValues(catalog, namespace, table) + .inc(metrics.scannedDataManifests().value()); + } + + if (metrics.skippedDataManifests() != null) { + scanSkippedDataManifests + .labelValues(catalog, namespace, table) + .inc(metrics.skippedDataManifests().value()); + } + + // File sizes + if (metrics.totalFileSizeInBytes() != null) { + scanTotalFileSizeBytes + .labelValues(catalog, namespace, table) + .inc(metrics.totalFileSizeInBytes().value()); + } + + if (metrics.totalDeleteFileSizeInBytes() != null) { + scanTotalDeleteFileSizeBytes + .labelValues(catalog, namespace, table) + .inc(metrics.totalDeleteFileSizeInBytes().value()); + } + + // Timing metrics + if (metrics.totalPlanningDuration() != null) { + observeDuration( + scanPlanningDuration.labelValues(catalog, namespace, table), + metrics.totalPlanningDuration().totalDuration()); + } + } + + private void reportCommitMetrics(String catalog, CommitReport report) { + String fullTableName = report.tableName(); + String namespace = extractNamespace(fullTableName); + String table = extractTableName(fullTableName); + String operation = report.operation() != null ? report.operation() : "unknown"; + + logger.debug( + "Recording commit metrics for catalog: {}, namespace: {}, table: {}, operation: {}", + catalog, + namespace, + table, + operation); + + commitsTotal.labelValues(catalog, namespace, table, operation).inc(); + + var metrics = report.commitMetrics(); + if (metrics == null) { + return; + } + + if (metrics.addedDataFiles() != null) { + commitAddedDataFiles + .labelValues(catalog, namespace, table, operation) + .inc(metrics.addedDataFiles().value()); + } + + if (metrics.removedDataFiles() != null) { + commitRemovedDataFiles + .labelValues(catalog, namespace, table, operation) + .inc(metrics.removedDataFiles().value()); + } + + if (metrics.addedDeleteFiles() != null) { + commitAddedDeleteFiles + .labelValues(catalog, namespace, table, operation) + .inc(metrics.addedDeleteFiles().value()); + } + + if (metrics.removedDeleteFiles() != null) { + commitRemovedDeleteFiles + .labelValues(catalog, namespace, table, operation) + .inc(metrics.removedDeleteFiles().value()); + } + + if (metrics.addedRecords() != null) { + commitAddedRecords + .labelValues(catalog, namespace, table, operation) + .inc(metrics.addedRecords().value()); + } + + if (metrics.removedRecords() != null) { + commitRemovedRecords + .labelValues(catalog, namespace, table, operation) + .inc(metrics.removedRecords().value()); + } + + if (metrics.addedEqualityDeletes() != null) { + commitAddedEqualityDeletes + .labelValues(catalog, namespace, table, operation) + .inc(metrics.addedEqualityDeletes().value()); + } + + if (metrics.totalFilesSizeInBytes() != null) { + commitTotalFilesSizeBytes + .labelValues(catalog, namespace, table, operation) + .inc(metrics.totalFilesSizeInBytes().value()); + } + + if (metrics.totalDuration() != null) { + observeDuration( + commitDuration.labelValues(catalog, namespace, table, operation), + metrics.totalDuration().totalDuration()); + } + } + + private void observeDuration(DistributionDataPoint dataPoint, java.time.Duration duration) { + if (duration != null) { + double seconds = duration.toNanos() / (double) TimeUnit.SECONDS.toNanos(1); + dataPoint.observe(seconds); + } + } + + /** + * Extracts the namespace from a full table name. Handles various formats: + * + *

    + *
  • "namespace.table" -> "namespace" + *
  • "db.schema.table" -> "db.schema" + *
  • "table" -> "default" + *
+ */ + private String extractNamespace(String fullTableName) { + if (fullTableName == null || fullTableName.isEmpty()) { + return "unknown"; + } + String[] parts = fullTableName.split("\\."); + if (parts.length > 1) { + return String.join(".", Arrays.copyOf(parts, parts.length - 1)); + } + return "default"; + } + + /** + * Extracts the table name from a full table name. Handles various formats: + * + *
    + *
  • "namespace.table" -> "table" + *
  • "db.schema.table" -> "table" + *
  • "table" -> "table" + *
+ */ + private String extractTableName(String fullTableName) { + if (fullTableName == null || fullTableName.isEmpty()) { + return "unknown"; + } + String[] parts = fullTableName.split("\\."); + return parts[parts.length - 1]; + } +} diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogAdapter.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogAdapter.java index fa3b204..4d01805 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogAdapter.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogAdapter.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; import com.altinity.ice.rest.catalog.internal.auth.Session; +import com.altinity.ice.rest.catalog.internal.metrics.PrometheusMetricsReporter; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -66,12 +67,18 @@ public class RESTCatalogAdapter implements RESTCatalogHandler { private final Catalog catalog; private final SupportsNamespaces asNamespaceCatalog; private final ViewCatalog asViewCatalog; + private final PrometheusMetricsReporter metricsReporter; public RESTCatalogAdapter(Catalog catalog) { + this(catalog, null); + } + + public RESTCatalogAdapter(Catalog catalog, PrometheusMetricsReporter metricsReporter) { this.catalog = catalog; this.asNamespaceCatalog = catalog instanceof SupportsNamespaces ? (SupportsNamespaces) catalog : null; this.asViewCatalog = catalog instanceof ViewCatalog ? (ViewCatalog) catalog : null; + this.metricsReporter = metricsReporter; } @Override @@ -236,9 +243,14 @@ public T handle( } case REPORT_METRICS: - // nothing to do here other than checking that we're getting the correct request - castRequest(ReportMetricsRequest.class, requestBody); - return null; + { + ReportMetricsRequest request = castRequest(ReportMetricsRequest.class, requestBody); + if (metricsReporter != null && request.report() != null) { + String catalogName = catalog.name(); + metricsReporter.report(catalogName, request.report()); + } + return null; + } case COMMIT_TRANSACTION: { From 3776ed6bf779e8dc89c9047094ef8afd76d54a09 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Thu, 4 Dec 2025 12:27:02 -0600 Subject: [PATCH 02/29] Moved hardcoded strings to IcebergMetricNames, added dashboard json file. --- .../grafana/iceberg-metrics-dashboard.json | 1724 +++++++++++++++++ .../internal/metrics/IcebergMetricNames.java | 163 ++ .../metrics/PrometheusMetricsReporter.java | 144 +- 3 files changed, 1961 insertions(+), 70 deletions(-) create mode 100644 examples/grafana/iceberg-metrics-dashboard.json create mode 100644 ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json new file mode 100644 index 0000000..f56b442 --- /dev/null +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -0,0 +1,1724 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": { + "type": "grafana", + "uid": "-- Grafana --" + }, + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 1, + "id": null, + "links": [], + "panels": [ + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 100, + "panels": [], + "title": "Overview", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 1 + }, + "id": 1, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(iceberg_scans_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "legendFormat": "Scans", + "refId": "A" + } + ], + "title": "Total Scans", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 1 + }, + "id": 2, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(iceberg_commits_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "legendFormat": "Commits", + "refId": "A" + } + ], + "title": "Total Commits", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 1 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 8, + "y": 1 + }, + "id": 3, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(iceberg_metrics_report_errors_total[$__range])) or vector(0)", + "legendFormat": "Errors", + "refId": "A" + } + ], + "title": "Metrics Errors", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "bytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 1 + }, + "id": 4, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(iceberg_scan_total_file_size_bytes{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "legendFormat": "Bytes Scanned", + "refId": "A" + } + ], + "title": "Data Scanned", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 1 + }, + "id": 5, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(iceberg_commit_added_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "legendFormat": "Records Added", + "refId": "A" + } + ], + "title": "Records Added", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [ + { + "options": { + "0": { + "color": "red", + "index": 1, + "text": "Inactive" + }, + "1": { + "color": "green", + "index": 0, + "text": "Active" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "red", + "value": null + }, + { + "color": "green", + "value": 1 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 1 + }, + "id": 6, + "options": { + "colorMode": "value", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "iceberg_metrics_reporter_active or vector(0)", + "legendFormat": "Reporter Status", + "refId": "A" + } + ], + "title": "Metrics Reporter", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ops" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 5 + }, + "id": 7, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_scans_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "{{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + } + ], + "title": "Scan Rate by Table", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "ops" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 5 + }, + "id": 8, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table, operation) (rate(iceberg_commits_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "{{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "refId": "A" + } + ], + "title": "Commit Rate by Table", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 13 + }, + "id": 101, + "panels": [], + "title": "Scan Metrics", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 14 + }, + "id": 10, + "options": { + "legend": { + "calcs": ["mean", "max"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.50, sum by (le, catalog, namespace, table) (rate(iceberg_scan_planning_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p50 - {{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le, catalog, namespace, table) (rate(iceberg_scan_planning_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p95 - {{catalog}}.{{namespace}}.{{table}}", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.99, sum by (le, catalog, namespace, table) (rate(iceberg_scan_planning_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p99 - {{catalog}}.{{namespace}}.{{table}}", + "refId": "C" + } + ], + "title": "Scan Planning Duration", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 14 + }, + "id": 11, + "options": { + "legend": { + "calcs": ["mean", "max"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.50, sum by (le, catalog, namespace, table) (rate(iceberg_scan_data_files_per_scan_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p50 - {{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le, catalog, namespace, table) (rate(iceberg_scan_data_files_per_scan_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p95 - {{catalog}}.{{namespace}}.{{table}}", + "refId": "B" + } + ], + "title": "Data Files Per Scan", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 22 + }, + "id": 12, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_scanned_data_manifests{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Scanned - {{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_skipped_data_manifests{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Skipped - {{catalog}}.{{namespace}}.{{table}}", + "refId": "B" + } + ], + "title": "Manifests Scanned vs Skipped", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "binBps" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 22 + }, + "id": 13, + "options": { + "legend": { + "calcs": ["mean", "sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_total_file_size_bytes{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "{{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + } + ], + "title": "Scan Throughput (bytes/sec)", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 30 + }, + "id": 102, + "panels": [], + "title": "Commit Metrics", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 31 + }, + "id": 20, + "options": { + "legend": { + "calcs": ["mean", "max"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.50, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p50 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p95 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.99, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "legendFormat": "p99 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "refId": "C" + } + ], + "title": "Commit Duration", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 31 + }, + "id": 21, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table, operation) (rate(iceberg_commit_added_data_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Added - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "-sum by (catalog, namespace, table, operation) (rate(iceberg_commit_removed_data_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Removed - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "refId": "B" + } + ], + "title": "Data Files Added/Removed", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 39 + }, + "id": 22, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_commit_added_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Added - {{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "-sum by (catalog, namespace, table) (rate(iceberg_commit_removed_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Removed - {{catalog}}.{{namespace}}.{{table}}", + "refId": "B" + } + ], + "title": "Records Added/Removed Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "binBps" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 39 + }, + "id": 23, + "options": { + "legend": { + "calcs": ["mean", "sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_commit_total_files_size_bytes{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "{{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + } + ], + "title": "Commit Throughput (bytes/sec)", + "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 47 + }, + "id": 103, + "panels": [], + "title": "Delete Files", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 48 + }, + "id": 30, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_positional_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Positional - {{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_equality_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Equality - {{catalog}}.{{namespace}}.{{table}}", + "refId": "B" + } + ], + "title": "Delete Files Scanned by Type", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 48 + }, + "id": 31, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (catalog, namespace, table) (rate(iceberg_commit_added_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Added - {{catalog}}.{{namespace}}.{{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "-sum by (catalog, namespace, table) (rate(iceberg_commit_removed_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", + "legendFormat": "Removed - {{catalog}}.{{namespace}}.{{table}}", + "refId": "B" + } + ], + "title": "Delete Files Added/Removed", + "type": "timeseries" + } + ], + "refresh": "30s", + "schemaVersion": 38, + "tags": ["iceberg", "data-lake", "rest-catalog"], + "templating": { + "list": [ + { + "current": {}, + "hide": 0, + "includeAll": false, + "label": "Datasource", + "multi": false, + "name": "datasource", + "options": [], + "query": "prometheus", + "queryValue": "", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "allValue": ".*", + "current": { + "selected": true, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values({__name__=~\"iceberg_.+\"}, catalog)", + "hide": 0, + "includeAll": true, + "label": "Catalog", + "multi": true, + "name": "catalog", + "options": [], + "query": { + "query": "label_values({__name__=~\"iceberg_.+\"}, catalog)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "type": "query" + }, + { + "allValue": ".*", + "current": { + "selected": true, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values({__name__=~\"iceberg_.+\", catalog=~\"$catalog\"}, namespace)", + "hide": 0, + "includeAll": true, + "label": "Namespace", + "multi": true, + "name": "namespace", + "options": [], + "query": { + "query": "label_values({__name__=~\"iceberg_.+\", catalog=~\"$catalog\"}, namespace)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "type": "query" + }, + { + "allValue": ".*", + "current": { + "selected": true, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values({__name__=~\"iceberg_.+\", catalog=~\"$catalog\", namespace=~\"$namespace\"}, table)", + "hide": 0, + "includeAll": true, + "label": "Table", + "multi": true, + "name": "table", + "options": [], + "query": { + "query": "label_values({__name__=~\"iceberg_.+\", catalog=~\"$catalog\", namespace=~\"$namespace\"}, table)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "type": "query" + } + ] + }, + "time": { + "from": "now-1h", + "to": "now" + }, + "timepicker": {}, + "timezone": "browser", + "title": "Iceberg REST Catalog Metrics", + "uid": "iceberg-rest-catalog-metrics", + "version": 1, + "weekStart": "" +} + diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java new file mode 100644 index 0000000..c9d6868 --- /dev/null +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java @@ -0,0 +1,163 @@ +/* + * Copyright (c) 2025 Altinity Inc and/or its affiliates. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.altinity.ice.rest.catalog.internal.metrics; + +/** Constants for Iceberg Prometheus metric names, help strings, and labels. */ +public final class IcebergMetricNames { + + private IcebergMetricNames() {} + + // ========================================================================== + // Labels + // ========================================================================== + + public static final String LABEL_CATALOG = "catalog"; + public static final String LABEL_NAMESPACE = "namespace"; + public static final String LABEL_TABLE = "table"; + public static final String LABEL_OPERATION = "operation"; + public static final String LABEL_TYPE = "type"; + + public static final String[] SCAN_LABELS = {LABEL_CATALOG, LABEL_NAMESPACE, LABEL_TABLE}; + public static final String[] COMMIT_LABELS = { + LABEL_CATALOG, LABEL_NAMESPACE, LABEL_TABLE, LABEL_OPERATION + }; + + // ========================================================================== + // Reporter Info Metrics + // ========================================================================== + + public static final String REPORTER_ACTIVE_NAME = "iceberg_metrics_reporter_active"; + public static final String REPORTER_ACTIVE_HELP = + "Iceberg metrics reporter status (value 1 means reporter is active)"; + + public static final String REPORT_ERRORS_NAME = "iceberg_metrics_report_errors_total"; + public static final String REPORT_ERRORS_HELP = + "Total number of errors while processing metrics reports"; + + // ========================================================================== + // Scan Metrics + // ========================================================================== + + public static final String SCANS_TOTAL_NAME = "iceberg_scans_total"; + public static final String SCANS_TOTAL_HELP = "Total number of Iceberg table scans"; + + public static final String SCAN_RESULT_DATA_FILES_NAME = "iceberg_scan_result_data_files_total"; + public static final String SCAN_RESULT_DATA_FILES_HELP = + "Total number of data files in scan results"; + + public static final String SCAN_RESULT_DELETE_FILES_NAME = + "iceberg_scan_result_delete_files_total"; + public static final String SCAN_RESULT_DELETE_FILES_HELP = + "Total number of delete files in scan results"; + + public static final String SCAN_INDEXED_DELETE_FILES_NAME = + "iceberg_scan_indexed_delete_files_total"; + public static final String SCAN_INDEXED_DELETE_FILES_HELP = + "Total number of indexed delete files in scan results"; + + public static final String SCAN_POSITIONAL_DELETE_FILES_NAME = + "iceberg_scan_positional_delete_files_total"; + public static final String SCAN_POSITIONAL_DELETE_FILES_HELP = + "Total number of positional delete files in scan results"; + + public static final String SCAN_EQUALITY_DELETE_FILES_NAME = + "iceberg_scan_equality_delete_files_total"; + public static final String SCAN_EQUALITY_DELETE_FILES_HELP = + "Total number of equality delete files in scan results"; + + public static final String SCAN_TOTAL_DATA_MANIFESTS_NAME = "iceberg_scan_total_data_manifests"; + public static final String SCAN_TOTAL_DATA_MANIFESTS_HELP = + "Total number of data manifests considered during scans"; + + public static final String SCAN_TOTAL_DELETE_MANIFESTS_NAME = + "iceberg_scan_total_delete_manifests"; + public static final String SCAN_TOTAL_DELETE_MANIFESTS_HELP = + "Total number of delete manifests considered during scans"; + + public static final String SCAN_SCANNED_DATA_MANIFESTS_NAME = + "iceberg_scan_scanned_data_manifests"; + public static final String SCAN_SCANNED_DATA_MANIFESTS_HELP = + "Total number of data manifests actually scanned"; + + public static final String SCAN_SKIPPED_DATA_MANIFESTS_NAME = + "iceberg_scan_skipped_data_manifests"; + public static final String SCAN_SKIPPED_DATA_MANIFESTS_HELP = + "Total number of data manifests skipped during scans"; + + public static final String SCAN_TOTAL_FILE_SIZE_BYTES_NAME = "iceberg_scan_total_file_size_bytes"; + public static final String SCAN_TOTAL_FILE_SIZE_BYTES_HELP = + "Total file size in bytes for scanned data files"; + + public static final String SCAN_TOTAL_DELETE_FILE_SIZE_BYTES_NAME = + "iceberg_scan_total_delete_file_size_bytes"; + public static final String SCAN_TOTAL_DELETE_FILE_SIZE_BYTES_HELP = + "Total file size in bytes for scanned delete files"; + + public static final String SCAN_PLANNING_DURATION_NAME = "iceberg_scan_planning_duration_seconds"; + public static final String SCAN_PLANNING_DURATION_HELP = "Duration of scan planning in seconds"; + + public static final String SCAN_DATA_FILES_PER_SCAN_NAME = "iceberg_scan_data_files_per_scan"; + public static final String SCAN_DATA_FILES_PER_SCAN_HELP = "Distribution of data files per scan"; + + // ========================================================================== + // Commit Metrics + // ========================================================================== + + public static final String COMMITS_TOTAL_NAME = "iceberg_commits_total"; + public static final String COMMITS_TOTAL_HELP = "Total number of Iceberg table commits"; + + public static final String COMMIT_ADDED_DATA_FILES_NAME = "iceberg_commit_added_data_files_total"; + public static final String COMMIT_ADDED_DATA_FILES_HELP = + "Total number of data files added in commits"; + + public static final String COMMIT_REMOVED_DATA_FILES_NAME = + "iceberg_commit_removed_data_files_total"; + public static final String COMMIT_REMOVED_DATA_FILES_HELP = + "Total number of data files removed in commits"; + + public static final String COMMIT_ADDED_DELETE_FILES_NAME = + "iceberg_commit_added_delete_files_total"; + public static final String COMMIT_ADDED_DELETE_FILES_HELP = + "Total number of delete files added in commits"; + + public static final String COMMIT_REMOVED_DELETE_FILES_NAME = + "iceberg_commit_removed_delete_files_total"; + public static final String COMMIT_REMOVED_DELETE_FILES_HELP = + "Total number of delete files removed in commits"; + + public static final String COMMIT_ADDED_RECORDS_NAME = "iceberg_commit_added_records_total"; + public static final String COMMIT_ADDED_RECORDS_HELP = "Total number of records added in commits"; + + public static final String COMMIT_REMOVED_RECORDS_NAME = "iceberg_commit_removed_records_total"; + public static final String COMMIT_REMOVED_RECORDS_HELP = + "Total number of records removed in commits"; + + public static final String COMMIT_ADDED_EQUALITY_DELETES_NAME = + "iceberg_commit_added_equality_deletes_total"; + public static final String COMMIT_ADDED_EQUALITY_DELETES_HELP = + "Total number of equality deletes added in commits"; + + public static final String COMMIT_TOTAL_FILES_SIZE_BYTES_NAME = + "iceberg_commit_total_files_size_bytes"; + public static final String COMMIT_TOTAL_FILES_SIZE_BYTES_HELP = + "Total size in bytes of files involved in commits"; + + public static final String COMMIT_DURATION_NAME = "iceberg_commit_duration_seconds"; + public static final String COMMIT_DURATION_HELP = "Duration of commit operations in seconds"; + + // ========================================================================== + // Histogram Buckets + // ========================================================================== + + /** Duration histogram buckets (in seconds) - suitable for typical Iceberg operations. */ + public static final double[] DURATION_BUCKETS = { + 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, 2.5, 5, 10, 30, 60 + }; +} diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java index 557623e..2807566 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/PrometheusMetricsReporter.java @@ -9,6 +9,8 @@ */ package com.altinity.ice.rest.catalog.internal.metrics; +import static com.altinity.ice.rest.catalog.internal.metrics.IcebergMetricNames.*; + import io.prometheus.metrics.core.datapoints.DistributionDataPoint; import io.prometheus.metrics.core.metrics.Counter; import io.prometheus.metrics.core.metrics.Histogram; @@ -16,6 +18,7 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.metrics.CommitReport; import org.apache.iceberg.metrics.MetricsReport; +import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.ScanReport; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,13 +27,21 @@ * A Prometheus-based metrics reporter for Iceberg operations. This reporter exposes Iceberg scan * and commit metrics via Prometheus, making them available at the /metrics endpoint. * + *

This class implements {@link MetricsReporter} to follow Iceberg's standard metrics reporting + * pattern, allowing it to be used both as a server-side reporter (receiving metrics via REST) and + * potentially as a client-side reporter via catalog configuration. + * *

This class uses a singleton pattern because Prometheus metrics can only be registered once per * JVM. * *

All duration metrics use Histograms instead of Summaries to allow aggregation across multiple * instances in distributed deployments. + * + * @see IcebergMetricNames for metric names and help strings + * @see Iceberg Metrics + * Reporting */ -public class PrometheusMetricsReporter { +public class PrometheusMetricsReporter implements MetricsReporter { private static final Logger logger = LoggerFactory.getLogger(PrometheusMetricsReporter.class); @@ -38,15 +49,6 @@ public class PrometheusMetricsReporter { private static volatile PrometheusMetricsReporter instance; private static final Object lock = new Object(); - // Common label names - private static final String[] SCAN_LABELS = {"catalog", "namespace", "table"}; - private static final String[] COMMIT_LABELS = {"catalog", "namespace", "table", "operation"}; - - // Duration histogram buckets (in seconds) - suitable for typical Iceberg operations - private static final double[] DURATION_BUCKETS = { - 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, 2.5, 5, 10, 30, 60 - }; - // Scan metrics counters private final Counter scansTotal; private final Counter scanResultDataFiles; @@ -82,8 +84,8 @@ public class PrometheusMetricsReporter { // Error counter private final Counter metricsReportErrors; - // Info metric to confirm reporter is active - private final Counter metricsReporterInfo; + // Status metric to confirm reporter is active + private final Counter metricsReporterActive; /** Returns the singleton instance of the metrics reporter. */ public static PrometheusMetricsReporter getInstance() { @@ -98,120 +100,117 @@ public static PrometheusMetricsReporter getInstance() { } private PrometheusMetricsReporter() { - // Info metric - incremented once to confirm reporter is active and visible in /metrics - this.metricsReporterInfo = - Counter.builder() - .name("iceberg_metrics_reporter_info") - .help("Iceberg metrics reporter info (value 1 means reporter is active)") - .register(); - this.metricsReporterInfo.inc(); // Make it visible immediately + // Status metric - incremented once to confirm reporter is active and visible in /metrics + this.metricsReporterActive = + Counter.builder().name(REPORTER_ACTIVE_NAME).help(REPORTER_ACTIVE_HELP).register(); + this.metricsReporterActive.inc(); // Make it visible immediately // Error counter this.metricsReportErrors = Counter.builder() - .name("iceberg_metrics_report_errors_total") - .help("Total number of errors while processing metrics reports") - .labelNames("type") + .name(REPORT_ERRORS_NAME) + .help(REPORT_ERRORS_HELP) + .labelNames(LABEL_TYPE) .register(); // Scan metrics this.scansTotal = Counter.builder() - .name("iceberg_scans_total") - .help("Total number of Iceberg table scans") + .name(SCANS_TOTAL_NAME) + .help(SCANS_TOTAL_HELP) .labelNames(SCAN_LABELS) .register(); this.scanResultDataFiles = Counter.builder() - .name("iceberg_scan_result_data_files_total") - .help("Total number of data files in scan results") + .name(SCAN_RESULT_DATA_FILES_NAME) + .help(SCAN_RESULT_DATA_FILES_HELP) .labelNames(SCAN_LABELS) .register(); this.scanResultDeleteFiles = Counter.builder() - .name("iceberg_scan_result_delete_files_total") - .help("Total number of delete files in scan results") + .name(SCAN_RESULT_DELETE_FILES_NAME) + .help(SCAN_RESULT_DELETE_FILES_HELP) .labelNames(SCAN_LABELS) .register(); this.scanIndexedDeleteFiles = Counter.builder() - .name("iceberg_scan_indexed_delete_files_total") - .help("Total number of indexed delete files in scan results") + .name(SCAN_INDEXED_DELETE_FILES_NAME) + .help(SCAN_INDEXED_DELETE_FILES_HELP) .labelNames(SCAN_LABELS) .register(); this.scanPositionalDeleteFiles = Counter.builder() - .name("iceberg_scan_positional_delete_files_total") - .help("Total number of positional delete files in scan results") + .name(SCAN_POSITIONAL_DELETE_FILES_NAME) + .help(SCAN_POSITIONAL_DELETE_FILES_HELP) .labelNames(SCAN_LABELS) .register(); this.scanEqualityDeleteFiles = Counter.builder() - .name("iceberg_scan_equality_delete_files_total") - .help("Total number of equality delete files in scan results") + .name(SCAN_EQUALITY_DELETE_FILES_NAME) + .help(SCAN_EQUALITY_DELETE_FILES_HELP) .labelNames(SCAN_LABELS) .register(); this.scanTotalDataManifests = Counter.builder() - .name("iceberg_scan_total_data_manifests") - .help("Total number of data manifests considered during scans") + .name(SCAN_TOTAL_DATA_MANIFESTS_NAME) + .help(SCAN_TOTAL_DATA_MANIFESTS_HELP) .labelNames(SCAN_LABELS) .register(); this.scanTotalDeleteManifests = Counter.builder() - .name("iceberg_scan_total_delete_manifests") - .help("Total number of delete manifests considered during scans") + .name(SCAN_TOTAL_DELETE_MANIFESTS_NAME) + .help(SCAN_TOTAL_DELETE_MANIFESTS_HELP) .labelNames(SCAN_LABELS) .register(); this.scanScannedDataManifests = Counter.builder() - .name("iceberg_scan_scanned_data_manifests") - .help("Total number of data manifests actually scanned") + .name(SCAN_SCANNED_DATA_MANIFESTS_NAME) + .help(SCAN_SCANNED_DATA_MANIFESTS_HELP) .labelNames(SCAN_LABELS) .register(); this.scanSkippedDataManifests = Counter.builder() - .name("iceberg_scan_skipped_data_manifests") - .help("Total number of data manifests skipped during scans") + .name(SCAN_SKIPPED_DATA_MANIFESTS_NAME) + .help(SCAN_SKIPPED_DATA_MANIFESTS_HELP) .labelNames(SCAN_LABELS) .register(); this.scanTotalFileSizeBytes = Counter.builder() - .name("iceberg_scan_total_file_size_bytes") - .help("Total file size in bytes for scanned data files") + .name(SCAN_TOTAL_FILE_SIZE_BYTES_NAME) + .help(SCAN_TOTAL_FILE_SIZE_BYTES_HELP) .labelNames(SCAN_LABELS) .register(); this.scanTotalDeleteFileSizeBytes = Counter.builder() - .name("iceberg_scan_total_delete_file_size_bytes") - .help("Total file size in bytes for scanned delete files") + .name(SCAN_TOTAL_DELETE_FILE_SIZE_BYTES_NAME) + .help(SCAN_TOTAL_DELETE_FILE_SIZE_BYTES_HELP) .labelNames(SCAN_LABELS) .register(); // Scan timing - using Histogram for aggregation across instances this.scanPlanningDuration = Histogram.builder() - .name("iceberg_scan_planning_duration_seconds") - .help("Duration of scan planning in seconds") + .name(SCAN_PLANNING_DURATION_NAME) + .help(SCAN_PLANNING_DURATION_HELP) .labelNames(SCAN_LABELS) .classicUpperBounds(DURATION_BUCKETS) .register(); this.scanDataFilesPerScan = Histogram.builder() - .name("iceberg_scan_data_files_per_scan") - .help("Distribution of data files per scan") + .name(SCAN_DATA_FILES_PER_SCAN_NAME) + .help(SCAN_DATA_FILES_PER_SCAN_HELP) .labelNames(SCAN_LABELS) .classicExponentialUpperBounds(1, 2, 10) .register(); @@ -219,72 +218,72 @@ private PrometheusMetricsReporter() { // Commit metrics this.commitsTotal = Counter.builder() - .name("iceberg_commits_total") - .help("Total number of Iceberg table commits") + .name(COMMITS_TOTAL_NAME) + .help(COMMITS_TOTAL_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitAddedDataFiles = Counter.builder() - .name("iceberg_commit_added_data_files_total") - .help("Total number of data files added in commits") + .name(COMMIT_ADDED_DATA_FILES_NAME) + .help(COMMIT_ADDED_DATA_FILES_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitRemovedDataFiles = Counter.builder() - .name("iceberg_commit_removed_data_files_total") - .help("Total number of data files removed in commits") + .name(COMMIT_REMOVED_DATA_FILES_NAME) + .help(COMMIT_REMOVED_DATA_FILES_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitAddedDeleteFiles = Counter.builder() - .name("iceberg_commit_added_delete_files_total") - .help("Total number of delete files added in commits") + .name(COMMIT_ADDED_DELETE_FILES_NAME) + .help(COMMIT_ADDED_DELETE_FILES_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitRemovedDeleteFiles = Counter.builder() - .name("iceberg_commit_removed_delete_files_total") - .help("Total number of delete files removed in commits") + .name(COMMIT_REMOVED_DELETE_FILES_NAME) + .help(COMMIT_REMOVED_DELETE_FILES_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitAddedRecords = Counter.builder() - .name("iceberg_commit_added_records_total") - .help("Total number of records added in commits") + .name(COMMIT_ADDED_RECORDS_NAME) + .help(COMMIT_ADDED_RECORDS_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitRemovedRecords = Counter.builder() - .name("iceberg_commit_removed_records_total") - .help("Total number of records removed in commits") + .name(COMMIT_REMOVED_RECORDS_NAME) + .help(COMMIT_REMOVED_RECORDS_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitAddedEqualityDeletes = Counter.builder() - .name("iceberg_commit_added_equality_deletes_total") - .help("Total number of equality deletes added in commits") + .name(COMMIT_ADDED_EQUALITY_DELETES_NAME) + .help(COMMIT_ADDED_EQUALITY_DELETES_HELP) .labelNames(COMMIT_LABELS) .register(); this.commitTotalFilesSizeBytes = Counter.builder() - .name("iceberg_commit_total_files_size_bytes") - .help("Total size in bytes of files involved in commits") + .name(COMMIT_TOTAL_FILES_SIZE_BYTES_NAME) + .help(COMMIT_TOTAL_FILES_SIZE_BYTES_HELP) .labelNames(COMMIT_LABELS) .register(); // Commit timing - using Histogram for aggregation across instances this.commitDuration = Histogram.builder() - .name("iceberg_commit_duration_seconds") - .help("Duration of commit operations in seconds") + .name(COMMIT_DURATION_NAME) + .help(COMMIT_DURATION_HELP) .labelNames(COMMIT_LABELS) .classicUpperBounds(DURATION_BUCKETS) .register(); @@ -299,6 +298,7 @@ private PrometheusMetricsReporter() { * @param report the metrics report to process */ public void report(String catalogName, MetricsReport report) { + logger.debug("Reporting metrics report: catalogName: {}, report: {}", catalogName, report); if (report == null) { return; } @@ -323,9 +323,13 @@ public void report(String catalogName, MetricsReport report) { /** * Report metrics from an Iceberg MetricsReport using default catalog name. * + *

This method implements {@link MetricsReporter#report(MetricsReport)}. + * * @param report the metrics report to process */ + @Override public void report(MetricsReport report) { + logger.debug("Reporting metrics report: {}", report); report(null, report); } From e7859cce4e07b5fded5815cfc1628a47e1909766 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Fri, 5 Dec 2025 13:14:18 -0600 Subject: [PATCH 03/29] Added prometheus HTTP metrics to track request duration and response errors. --- .../grafana/iceberg-metrics-dashboard.json | 666 ++++++++++++++++++ .../catalog/internal/metrics/HttpMetrics.java | 151 ++++ .../internal/metrics/IcebergMetricNames.java | 32 + .../internal/rest/RESTCatalogServlet.java | 126 ++-- 4 files changed, 919 insertions(+), 56 deletions(-) create mode 100644 ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index f56b442..806f00e 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -1602,6 +1602,672 @@ ], "title": "Delete Files Added/Removed", "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 56 + }, + "id": 104, + "panels": [], + "title": "HTTP/REST Endpoint Metrics", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "reqps" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 6, + "x": 0, + "y": 57 + }, + "id": 40, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(rate(iceberg_http_requests_total[$__rate_interval]))", + "legendFormat": "Requests/sec", + "refId": "A" + } + ], + "title": "Request Rate", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "yellow", + "value": 0.01 + }, + { + "color": "red", + "value": 0.05 + } + ] + }, + "unit": "percentunit" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 6, + "x": 6, + "y": 57 + }, + "id": 41, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(rate(iceberg_http_responses_total{status_class=~\"4xx|5xx\"}[$__rate_interval])) / sum(rate(iceberg_http_responses_total[$__rate_interval])) or vector(0)", + "legendFormat": "Error Rate", + "refId": "A" + } + ], + "title": "Error Rate (4xx+5xx)", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 6, + "x": 12, + "y": 57 + }, + "id": 42, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "iceberg_http_requests_in_flight or vector(0)", + "legendFormat": "In Flight", + "refId": "A" + } + ], + "title": "Requests In Flight", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "yellow", + "value": 0.1 + }, + { + "color": "red", + "value": 0.5 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 6, + "x": 18, + "y": 57 + }, + "id": 43, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le) (rate(iceberg_http_request_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p95 Latency", + "refId": "A" + } + ], + "title": "p95 Latency", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "reqps" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 61 + }, + "id": 44, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(rate(iceberg_http_requests_total[$__rate_interval]))", + "legendFormat": "Requests/sec", + "refId": "A" + } + ], + "title": "Request Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "reqps" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*4xx.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "yellow", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": ".*5xx.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 61 + }, + "id": 45, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (status_class) (rate(iceberg_http_responses_total[$__rate_interval]))", + "legendFormat": "{{status_class}}", + "refId": "A" + } + ], + "title": "Response Rate by Status", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 69 + }, + "id": 46, + "options": { + "legend": { + "calcs": ["mean", "max"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.50, sum by (le) (rate(iceberg_http_request_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p50", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le) (rate(iceberg_http_request_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p95", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.99, sum by (le) (rate(iceberg_http_request_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p99", + "refId": "C" + } + ], + "title": "Request Duration", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "reqps" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 69 + }, + "id": 47, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (route, status_class) (rate(iceberg_http_responses_total{status_class=~\"4xx|5xx\"}[$__rate_interval])) > 0", + "legendFormat": "{{route}} ({{status_class}})", + "refId": "A" + } + ], + "title": "Errors by Route", + "type": "timeseries" } ], "refresh": "30s", diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java new file mode 100644 index 0000000..1e913c5 --- /dev/null +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java @@ -0,0 +1,151 @@ +/* + * Copyright (c) 2025 Altinity Inc and/or its affiliates. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.altinity.ice.rest.catalog.internal.metrics; + +import static com.altinity.ice.rest.catalog.internal.metrics.IcebergMetricNames.*; + +import io.prometheus.metrics.core.metrics.Counter; +import io.prometheus.metrics.core.metrics.Gauge; +import io.prometheus.metrics.core.metrics.Histogram; +import java.util.concurrent.TimeUnit; + +/** + * Prometheus metrics for HTTP/REST endpoint monitoring. + * + *

This class uses a singleton pattern because Prometheus metrics can only be registered once per + * JVM. + */ +public class HttpMetrics { + + // Singleton instance + private static volatile HttpMetrics instance; + private static final Object lock = new Object(); + + private final Counter requestsTotal; + private final Counter responsesTotal; + private final Histogram requestDuration; + private final Gauge requestsInFlight; + + /** Returns the singleton instance of the HTTP metrics. */ + public static HttpMetrics getInstance() { + if (instance == null) { + synchronized (lock) { + if (instance == null) { + instance = new HttpMetrics(); + } + } + } + return instance; + } + + private HttpMetrics() { + this.requestsTotal = + Counter.builder() + .name(HTTP_REQUESTS_TOTAL_NAME) + .help(HTTP_REQUESTS_TOTAL_HELP) + .labelNames(HTTP_REQUEST_LABELS) + .register(); + + this.responsesTotal = + Counter.builder() + .name(HTTP_RESPONSES_TOTAL_NAME) + .help(HTTP_RESPONSES_TOTAL_HELP) + .labelNames(HTTP_RESPONSE_LABELS) + .register(); + + this.requestDuration = + Histogram.builder() + .name(HTTP_REQUEST_DURATION_NAME) + .help(HTTP_REQUEST_DURATION_HELP) + .labelNames(HTTP_REQUEST_LABELS) + .classicUpperBounds(HTTP_DURATION_BUCKETS) + .register(); + + this.requestsInFlight = + Gauge.builder() + .name(HTTP_REQUESTS_IN_FLIGHT_NAME) + .help(HTTP_REQUESTS_IN_FLIGHT_HELP) + .register(); + } + + /** + * Record the start of a request. Call this at the beginning of request handling. + * + * @param method HTTP method (GET, POST, etc.) + * @param route Route name (e.g., LOAD_TABLE, LIST_NAMESPACES) + */ + public void recordRequestStart(String method, String route) { + requestsTotal.labelValues(method, route).inc(); + requestsInFlight.inc(); + } + + /** + * Record the completion of a request. Call this at the end of request handling. + * + * @param method HTTP method (GET, POST, etc.) + * @param route Route name (e.g., LOAD_TABLE, LIST_NAMESPACES) + * @param statusCode HTTP status code (200, 404, 500, etc.) + * @param startTimeNanos Start time from System.nanoTime() + */ + public void recordRequestEnd(String method, String route, int statusCode, long startTimeNanos) { + requestsInFlight.dec(); + + // Record duration + double durationSeconds = + (System.nanoTime() - startTimeNanos) / (double) TimeUnit.SECONDS.toNanos(1); + requestDuration.labelValues(method, route).observe(durationSeconds); + + // Record response by status code + responsesTotal.labelValues(method, route, Integer.toString(statusCode)).inc(); + } + + /** + * Helper class for timing requests using try-with-resources. + * + *

Usage: + * + *

{@code
+   * try (var timer = httpMetrics.startRequest("GET", "LOAD_TABLE")) {
+   *   // handle request
+   *   timer.setStatusCode(200);
+   * }
+   * }
+ */ + public RequestTimer startRequest(String method, String route) { + return new RequestTimer(this, method, route); + } + + /** Auto-closeable timer for request duration tracking. */ + public static class RequestTimer implements AutoCloseable { + private final HttpMetrics metrics; + private final String method; + private final String route; + private final long startTimeNanos; + private int statusCode = 200; + + RequestTimer(HttpMetrics metrics, String method, String route) { + this.metrics = metrics; + this.method = method; + this.route = route; + this.startTimeNanos = System.nanoTime(); + metrics.recordRequestStart(method, route); + } + + /** Set the status code before closing. Default is 200. */ + public void setStatusCode(int statusCode) { + this.statusCode = statusCode; + } + + @Override + public void close() { + metrics.recordRequestEnd(method, route, statusCode, startTimeNanos); + } + } +} diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java index c9d6868..997e02a 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java @@ -160,4 +160,36 @@ private IcebergMetricNames() {} public static final double[] DURATION_BUCKETS = { 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, 2.5, 5, 10, 30, 60 }; + + // ========================================================================== + // HTTP/REST Endpoint Metrics + // ========================================================================== + + public static final String LABEL_METHOD = "method"; + public static final String LABEL_ROUTE = "route"; + public static final String LABEL_STATUS_CLASS = "status_class"; + + public static final String[] HTTP_REQUEST_LABELS = {LABEL_METHOD, LABEL_ROUTE}; + public static final String[] HTTP_RESPONSE_LABELS = { + LABEL_METHOD, LABEL_ROUTE, LABEL_STATUS_CLASS + }; + + public static final String HTTP_REQUESTS_TOTAL_NAME = "iceberg_http_requests_total"; + public static final String HTTP_REQUESTS_TOTAL_HELP = "Total number of HTTP requests"; + + public static final String HTTP_REQUEST_DURATION_NAME = "iceberg_http_request_duration_seconds"; + public static final String HTTP_REQUEST_DURATION_HELP = "HTTP request duration in seconds"; + + public static final String HTTP_RESPONSES_TOTAL_NAME = "iceberg_http_responses_total"; + public static final String HTTP_RESPONSES_TOTAL_HELP = + "Total number of HTTP responses by status class"; + + public static final String HTTP_REQUESTS_IN_FLIGHT_NAME = "iceberg_http_requests_in_flight"; + public static final String HTTP_REQUESTS_IN_FLIGHT_HELP = + "Number of HTTP requests currently being processed"; + + /** HTTP request duration buckets (in seconds) - suitable for REST API calls. */ + public static final double[] HTTP_DURATION_BUCKETS = { + 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, 2.5, 5, 10 + }; } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java index 3f8d0d4..afb227d 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java @@ -19,6 +19,7 @@ package com.altinity.ice.rest.catalog.internal.rest; import com.altinity.ice.rest.catalog.internal.auth.Session; +import com.altinity.ice.rest.catalog.internal.metrics.HttpMetrics; import jakarta.servlet.http.HttpServlet; import jakarta.servlet.http.HttpServletRequest; import jakarta.servlet.http.HttpServletResponse; @@ -72,9 +73,11 @@ public class RESTCatalogServlet extends HttpServlet { .buildOrThrow(); private final RESTCatalogHandler restCatalogAdapter; + private final HttpMetrics httpMetrics; public RESTCatalogServlet(RESTCatalogHandler restCatalogAdapter) { this.restCatalogAdapter = restCatalogAdapter; + this.httpMetrics = HttpMetrics.getInstance(); } protected void handle(HttpServletRequest request, HttpServletResponse response) @@ -84,70 +87,81 @@ protected void handle(HttpServletRequest request, HttpServletResponse response) Pair> routeContext = Route.from(method, path); if (routeContext == null) { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST); - var res = - ErrorResponse.builder() - .responseCode(400) - .withType("BadRequestException") - .withMessage(String.format("No route for %s %s", method, path)) - .build(); - RESTObjectMapper.mapper().writeValue(response.getWriter(), res); + // Track unknown route requests + try (var timer = httpMetrics.startRequest(method.name(), "UNKNOWN")) { + timer.setStatusCode(HttpServletResponse.SC_BAD_REQUEST); + response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + var res = + ErrorResponse.builder() + .responseCode(400) + .withType("BadRequestException") + .withMessage(String.format("No route for %s %s", method, path)) + .build(); + RESTObjectMapper.mapper().writeValue(response.getWriter(), res); + } return; } - Session session = Session.from(request); - String userToLog = ""; - if (session != null) { - userToLog = "@" + session.uid() + " "; - } - logger.info("{}{} {}", userToLog, method, path); - Route route = routeContext.first(); - Map pathParams = routeContext.second(); - - // FIXME: this should be in RESTCatalogAdapter, not here - Object requestBody = null; - if (route.requestClass() != null) { - requestBody = RESTObjectMapper.mapper().readValue(request.getReader(), route.requestClass()); - } else if (route == Route.TOKENS) { - requestBody = RESTUtil.decodeFormData(CharStreams.toString(request.getReader())); - } - Map queryParams = - request.getParameterMap().entrySet().stream() - .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue()[0])); - - Map params = - ImmutableMap.builder().putAll(pathParams).putAll(queryParams).build(); - - RESTResponse responseBody; - try { - responseBody = - restCatalogAdapter.handle(session, route, params, requestBody, route.responseClass()); - } catch (Exception e) { - ErrorResponse error = - ErrorResponse.builder() - .responseCode(STATUS_CODE_BY_EXCEPTION.getOrDefault(e.getClass(), 500)) - .withType(e.getClass().getSimpleName()) - .withMessage(e.getMessage()) - .build(); - - if (error.code() < 500) { - logger.warn("{}{} {}: {}", userToLog, method, path, e.getMessage()); - } else { - logger.error("{}{} {}", userToLog, method, path, e); + // Track request with metrics + try (var timer = httpMetrics.startRequest(method.name(), route.name())) { + Session session = Session.from(request); + String userToLog = ""; + if (session != null) { + userToLog = "@" + session.uid() + " "; } + logger.info("{}{} {}", userToLog, method, path); - response.setStatus(error.code()); - response.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); - RESTObjectMapper.mapper().writeValue(response.getWriter(), error); - return; - } + Map pathParams = routeContext.second(); - response.setStatus(HttpServletResponse.SC_OK); - response.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); - if (responseBody != null) { - RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody); + // FIXME: this should be in RESTCatalogAdapter, not here + Object requestBody = null; + if (route.requestClass() != null) { + requestBody = + RESTObjectMapper.mapper().readValue(request.getReader(), route.requestClass()); + } else if (route == Route.TOKENS) { + requestBody = RESTUtil.decodeFormData(CharStreams.toString(request.getReader())); + } + + Map queryParams = + request.getParameterMap().entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue()[0])); + + Map params = + ImmutableMap.builder().putAll(pathParams).putAll(queryParams).build(); + + RESTResponse responseBody; + try { + responseBody = + restCatalogAdapter.handle(session, route, params, requestBody, route.responseClass()); + } catch (Exception e) { + ErrorResponse error = + ErrorResponse.builder() + .responseCode(STATUS_CODE_BY_EXCEPTION.getOrDefault(e.getClass(), 500)) + .withType(e.getClass().getSimpleName()) + .withMessage(e.getMessage()) + .build(); + + if (error.code() < 500) { + logger.warn("{}{} {}: {}", userToLog, method, path, e.getMessage()); + } else { + logger.error("{}{} {}", userToLog, method, path, e); + } + + timer.setStatusCode(error.code()); + response.setStatus(error.code()); + response.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); + RESTObjectMapper.mapper().writeValue(response.getWriter(), error); + return; + } + + timer.setStatusCode(HttpServletResponse.SC_OK); + response.setStatus(HttpServletResponse.SC_OK); + response.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); + if (responseBody != null) { + RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody); + } } } From 6f908f4bbf2a4c0b93df2ff6bd5ba480c0692172 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Fri, 5 Dec 2025 13:23:11 -0600 Subject: [PATCH 04/29] Removed extra comments. --- .../catalog/internal/metrics/HttpMetrics.java | 38 ------------------- 1 file changed, 38 deletions(-) diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java index 1e913c5..ae060e9 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java @@ -16,15 +16,8 @@ import io.prometheus.metrics.core.metrics.Histogram; import java.util.concurrent.TimeUnit; -/** - * Prometheus metrics for HTTP/REST endpoint monitoring. - * - *

This class uses a singleton pattern because Prometheus metrics can only be registered once per - * JVM. - */ public class HttpMetrics { - // Singleton instance private static volatile HttpMetrics instance; private static final Object lock = new Object(); @@ -33,7 +26,6 @@ public class HttpMetrics { private final Histogram requestDuration; private final Gauge requestsInFlight; - /** Returns the singleton instance of the HTTP metrics. */ public static HttpMetrics getInstance() { if (instance == null) { synchronized (lock) { @@ -75,54 +67,24 @@ private HttpMetrics() { .register(); } - /** - * Record the start of a request. Call this at the beginning of request handling. - * - * @param method HTTP method (GET, POST, etc.) - * @param route Route name (e.g., LOAD_TABLE, LIST_NAMESPACES) - */ public void recordRequestStart(String method, String route) { requestsTotal.labelValues(method, route).inc(); requestsInFlight.inc(); } - /** - * Record the completion of a request. Call this at the end of request handling. - * - * @param method HTTP method (GET, POST, etc.) - * @param route Route name (e.g., LOAD_TABLE, LIST_NAMESPACES) - * @param statusCode HTTP status code (200, 404, 500, etc.) - * @param startTimeNanos Start time from System.nanoTime() - */ public void recordRequestEnd(String method, String route, int statusCode, long startTimeNanos) { requestsInFlight.dec(); - // Record duration double durationSeconds = (System.nanoTime() - startTimeNanos) / (double) TimeUnit.SECONDS.toNanos(1); requestDuration.labelValues(method, route).observe(durationSeconds); - - // Record response by status code responsesTotal.labelValues(method, route, Integer.toString(statusCode)).inc(); } - /** - * Helper class for timing requests using try-with-resources. - * - *

Usage: - * - *

{@code
-   * try (var timer = httpMetrics.startRequest("GET", "LOAD_TABLE")) {
-   *   // handle request
-   *   timer.setStatusCode(200);
-   * }
-   * }
- */ public RequestTimer startRequest(String method, String route) { return new RequestTimer(this, method, route); } - /** Auto-closeable timer for request duration tracking. */ public static class RequestTimer implements AutoCloseable { private final HttpMetrics metrics; private final String method; From 32f5adcbab9d0be5fee7cb82a864d03bc2227da0 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Fri, 5 Dec 2025 17:03:33 -0600 Subject: [PATCH 05/29] Added docker compose for ElasticMQ and configuration to test SQS locally. --- examples/s3watch/test/README.md | 24 +++++++++++++ examples/s3watch/test/docker-compose.yaml | 9 +++++ examples/s3watch/test/elasticmq.conf | 44 +++++++++++++++++++++++ 3 files changed, 77 insertions(+) create mode 100644 examples/s3watch/test/README.md create mode 100644 examples/s3watch/test/docker-compose.yaml create mode 100644 examples/s3watch/test/elasticmq.conf diff --git a/examples/s3watch/test/README.md b/examples/s3watch/test/README.md new file mode 100644 index 0000000..f1f44de --- /dev/null +++ b/examples/s3watch/test/README.md @@ -0,0 +1,24 @@ +### Local testing + +The `ice insert --watch` can also be tested locally with a ElasticMQ server (which is SQS compatible) + +Start the ElasticMQ server +`docker compose up` + +Start ice in insert mode. +`insert flowers.iris -p --no-copy --skip-duplicates s3://bucket1/flowers/iris/external-data/ --watch="http://localhost:9324/000000000000/s3-events"` + +Insert a S3 notification message(test) to ElasticMQ +``` +export AWS_ACCESS_KEY_ID=x +export AWS_SECRET_ACCESS_KEY=x +export AWS_REGION=us-east-1 + +aws --endpoint-url http://localhost:9324 sqs send-message \ + --queue-url http://localhost:9324/000000000000/s3-events \ + --message-body '{"Records":[{"eventName":"ObjectCreated:Put","s3":{"bucket":{"name":"bucket1"},"object":{"key":"flowers/iris/external-data/iris.parquet"}}}]}' +{ + "MD5OfMessageBody": "0ca3828dbdd1604d4b22fdfcb1226996", + "MessageId": "570bfd40-c0be-49f8-8119-25b74aad0894" +} +``` diff --git a/examples/s3watch/test/docker-compose.yaml b/examples/s3watch/test/docker-compose.yaml new file mode 100644 index 0000000..601ed14 --- /dev/null +++ b/examples/s3watch/test/docker-compose.yaml @@ -0,0 +1,9 @@ +services: + elasticmq: + image: softwaremill/elasticmq-native:1.6.9 + restart: unless-stopped + ports: + - '9324:9324' # SQS API + - '9325:9325' # Web UI + volumes: + - ./elasticmq.conf:/opt/elasticmq.conf:ro diff --git a/examples/s3watch/test/elasticmq.conf b/examples/s3watch/test/elasticmq.conf new file mode 100644 index 0000000..afee5de --- /dev/null +++ b/examples/s3watch/test/elasticmq.conf @@ -0,0 +1,44 @@ +include classpath("application.conf") + +node-address { + protocol = http + host = "*" + port = 9324 + context-path = "" +} + +rest-sqs { + enabled = true + bind-port = 9324 + bind-hostname = "0.0.0.0" + sqs-limits = strict +} + +rest-stats { + enabled = true + bind-port = 9325 + bind-hostname = "0.0.0.0" +} + +queues { + s3-events { + defaultVisibilityTimeout = 30 seconds + delay = 0 seconds + receiveMessageWait = 0 seconds + deadLettersQueue { + name = "s3-events-dlq" + maxReceiveCount = 3 + } + } + s3-events-dlq { + defaultVisibilityTimeout = 30 seconds + delay = 0 seconds + receiveMessageWait = 0 seconds + } +} + +aws { + region = elasticmq + accountId = 000000000000 +} + From a6918a3a0c96779165d93d5d98b4e5e99c2e3864 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Sat, 6 Dec 2025 20:26:18 -0600 Subject: [PATCH 06/29] Added InsertWatchMetrics and updated grafana dashboard. --- .../grafana/iceberg-metrics-dashboard.json | 860 +++++++++++++++++- examples/s3watch/test/docker-compose.yaml | 2 +- .../main/java/com/altinity/ice/cli/Main.java | 11 +- .../ice/cli/internal/cmd/InsertWatch.java | 119 ++- .../internal/metrics/InsertWatchMetrics.java | 263 ++++++ 5 files changed, 1247 insertions(+), 8 deletions(-) create mode 100644 ice/src/main/java/com/altinity/ice/cli/internal/metrics/InsertWatchMetrics.java diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index 806f00e..8a378b1 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -2268,11 +2268,785 @@ ], "title": "Errors by Route", "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 77 + }, + "id": 200, + "panels": [], + "title": "InsertWatch (S3 Watch) Metrics", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 5, + "x": 0, + "y": 78 + }, + "id": 202, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_watch_messages_received_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__range])) or vector(0)", + "legendFormat": "Messages", + "refId": "A" + } + ], + "title": "Messages Received", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 5, + "x": 5, + "y": 78 + }, + "id": 203, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_watch_files_inserted_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__range])) or vector(0)", + "legendFormat": "Files", + "refId": "A" + } + ], + "title": "Files Inserted", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 5, + "x": 10, + "y": 78 + }, + "id": 204, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_watch_transactions_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__range])) or vector(0)", + "legendFormat": "Transactions", + "refId": "A" + } + ], + "title": "Total Transactions", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 1 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 5, + "x": 15, + "y": 78 + }, + "id": 205, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_watch_transactions_failed_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__range])) or vector(0)", + "legendFormat": "Failed", + "refId": "A" + } + ], + "title": "Failed Transactions", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 1 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 78 + }, + "id": 206, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_watch_retry_attempts_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__range])) or vector(0)", + "legendFormat": "Retries", + "refId": "A" + } + ], + "title": "Retry Attempts", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 82 + }, + "id": 210, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_events_matched_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "Matched - {{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_events_not_matched_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "Not Matched - {{table}}", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_events_skipped_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "Skipped - {{table}}", + "refId": "C" + } + ], + "title": "Events by Match Status", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 82 + }, + "id": 211, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_files_inserted_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "{{table}}", + "refId": "A" + } + ], + "title": "Files Inserted Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*Failed.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 90 + }, + "id": 212, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_transactions_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "Success - {{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_transactions_failed_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "Failed - {{table}}", + "refId": "B" + } + ], + "title": "Transaction Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "normal" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*SQS Receive.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "orange", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": ".*SQS Delete.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "yellow", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": ".*Parse.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 90 + }, + "id": 213, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_sqs_receive_errors_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "SQS Receive - {{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_sqs_delete_errors_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "SQS Delete - {{table}}", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table, queue) (rate(ice_watch_message_parse_errors_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", + "legendFormat": "Parse Errors - {{table}}", + "refId": "C" + } + ], + "title": "Errors Rate", + "type": "timeseries" } ], "refresh": "30s", "schemaVersion": 38, - "tags": ["iceberg", "data-lake", "rest-catalog"], + "tags": ["iceberg", "data-lake", "rest-catalog", "s3-watch"], "templating": { "list": [ { @@ -2373,6 +3147,90 @@ "skipUrlSync": false, "sort": 1, "type": "query" + }, + { + "allValue": ".*", + "current": { + "selected": true, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values({__name__=~\"ice_watch_.+\"}, table)", + "hide": 0, + "includeAll": true, + "label": "Watch Table", + "multi": true, + "name": "watch_table", + "options": [], + "query": { + "query": "label_values({__name__=~\"ice_watch_.+\"}, table)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "type": "query" + }, + { + "allValue": ".*", + "current": { + "selected": true, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values({__name__=~\"ice_watch_.+\", table=~\"$watch_table\"}, queue)", + "hide": 0, + "includeAll": true, + "label": "Watch Queue", + "multi": true, + "name": "watch_queue", + "options": [], + "query": { + "query": "label_values({__name__=~\"ice_watch_.+\", table=~\"$watch_table\"}, queue)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "type": "query" + }, + { + "allValue": ".*", + "current": { + "selected": true, + "text": "All", + "value": "$__all" + }, + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "definition": "label_values({__name__=~\"ice_watch_.+\"}, instance)", + "hide": 0, + "includeAll": true, + "label": "Watch Instance", + "multi": true, + "name": "watch_instance", + "options": [], + "query": { + "query": "label_values({__name__=~\"ice_watch_.+\"}, instance)", + "refId": "StandardVariableQuery" + }, + "refresh": 2, + "regex": "", + "skipUrlSync": false, + "sort": 1, + "type": "query" } ] }, diff --git a/examples/s3watch/test/docker-compose.yaml b/examples/s3watch/test/docker-compose.yaml index 601ed14..3bf72c4 100644 --- a/examples/s3watch/test/docker-compose.yaml +++ b/examples/s3watch/test/docker-compose.yaml @@ -1,6 +1,6 @@ services: elasticmq: - image: softwaremill/elasticmq-native:1.6.9 + image: softwaremill/elasticmq-native:1.6.15 restart: unless-stopped ports: - '9324:9324' # SQS API diff --git a/ice/src/main/java/com/altinity/ice/cli/Main.java b/ice/src/main/java/com/altinity/ice/cli/Main.java index f6143ad..480ebcb 100644 --- a/ice/src/main/java/com/altinity/ice/cli/Main.java +++ b/ice/src/main/java/com/altinity/ice/cli/Main.java @@ -420,6 +420,7 @@ void insert( if (!watchMode) { Insert.run(catalog, tableId, dataFiles, options); } else { + boolean metricsEnabled = false; if (!Strings.isNullOrEmpty(watchDebugAddr)) { JvmMetrics.builder().register(); @@ -432,10 +433,18 @@ void insert( throw new RuntimeException(e); // TODO: find a better one } logger.info("Serving http://{}/{metrics,healtz,livez,readyz}", debugHostAndPort); + metricsEnabled = true; } InsertWatch.run( - catalog, tableId, dataFiles, watch, watchFireOnce, createTableIfNotExists, options); + catalog, + tableId, + dataFiles, + watch, + watchFireOnce, + createTableIfNotExists, + options, + metricsEnabled); } } } diff --git a/ice/src/main/java/com/altinity/ice/cli/internal/cmd/InsertWatch.java b/ice/src/main/java/com/altinity/ice/cli/internal/cmd/InsertWatch.java index bfad50e..8059140 100644 --- a/ice/src/main/java/com/altinity/ice/cli/internal/cmd/InsertWatch.java +++ b/ice/src/main/java/com/altinity/ice/cli/internal/cmd/InsertWatch.java @@ -9,11 +9,13 @@ */ package com.altinity.ice.cli.internal.cmd; +import com.altinity.ice.cli.internal.metrics.InsertWatchMetrics; import com.altinity.ice.internal.io.Matcher; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; +import java.net.URI; import java.net.URLDecoder; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -31,6 +33,7 @@ import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.SqsClientBuilder; import software.amazon.awssdk.services.sqs.model.BatchResultErrorEntry; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; @@ -52,6 +55,27 @@ public static void run( boolean createTableIfNotExists, Insert.Options options) throws IOException, InterruptedException { + run( + catalog, + nsTable, + input, + sqsQueueURL, + terminateAfterOneBatch, + createTableIfNotExists, + options, + false); + } + + public static void run( + RESTCatalog catalog, + TableIdentifier nsTable, + String[] input, + String sqsQueueURL, + boolean terminateAfterOneBatch, + boolean createTableIfNotExists, + Insert.Options options, + boolean metricsEnabled) + throws IOException, InterruptedException { if (!options.noCopy() || !options.skipDuplicates()) { throw new IllegalArgumentException( @@ -63,8 +87,14 @@ public static void run( } var matchers = Arrays.stream(input).map(Matcher::from).toList(); + logger.info("Watching for files matching: {}", Arrays.toString(input)); - final SqsClient sqs = SqsClient.builder().build(); + // Initialize metrics if enabled + InsertWatchMetrics metrics = metricsEnabled ? InsertWatchMetrics.getInstance() : null; + String tableLabel = nsTable.toString(); + String queueLabel = sqsQueueURL; + + final SqsClient sqs = buildSqsClient(sqsQueueURL); ReceiveMessageRequest req = ReceiveMessageRequest.builder() .queueUrl(sqsQueueURL) @@ -94,6 +124,10 @@ public static void run( var messages = sqs.receiveMessage(req).messages(); batch.addAll(messages); } catch (SdkException e) { + if (metrics != null) { + metrics.recordSqsReceiveError(tableLabel, queueLabel); + metrics.recordRetryAttempt(tableLabel, queueLabel); + } if (!e.retryable()) { throw e; // TODO: should we really? } @@ -112,17 +146,28 @@ public static void run( batch.addAll(tailMessages); } while (!tailMessages.isEmpty() && batch.size() < maxBatchSize); + if (metrics != null) { + metrics.recordMessagesReceived(tableLabel, queueLabel, batch.size()); + } + logger.info("Processing {} message(s)", batch.size()); // FIXME: handle files not found - var insertBatch = filter(batch, matchers); + var insertBatch = filter(batch, matchers, metrics, tableLabel, queueLabel); if (!insertBatch.isEmpty()) { logger.info("Inserting {}", insertBatch); try { Insert.run(catalog, nsTable, insertBatch.toArray(String[]::new), options); + if (metrics != null) { + metrics.recordFilesInserted(tableLabel, queueLabel, insertBatch.size()); + metrics.recordTransactionSuccess(tableLabel, queueLabel); + } } catch (NoSuchTableException e) { if (!createTableIfNotExists) { + if (metrics != null) { + metrics.recordTransactionFailed(tableLabel, queueLabel); + } throw e; } boolean retryInsert = true; @@ -139,6 +184,9 @@ public static void run( null); } catch (NotFoundException nfe) { if (!options.ignoreNotFound()) { + if (metrics != null) { + metrics.recordTransactionFailed(tableLabel, queueLabel); + } throw nfe; } logger.info("Table not created ({} don't exist)", insertBatch); @@ -146,16 +194,24 @@ public static void run( } if (retryInsert) { Insert.run(catalog, nsTable, insertBatch.toArray(String[]::new), options); + if (metrics != null) { + metrics.recordFilesInserted(tableLabel, queueLabel, insertBatch.size()); + metrics.recordTransactionSuccess(tableLabel, queueLabel); + } } } } - confirmProcessed(sqs, sqsQueueURL, batch); + confirmProcessed(sqs, sqsQueueURL, batch, metrics, tableLabel, queueLabel); } catch (InterruptedException e) { // terminate Thread.currentThread().interrupt(); throw new InterruptedException(); } catch (Exception e) { + if (metrics != null) { + metrics.recordTransactionFailed(tableLabel, queueLabel); + metrics.recordRetryAttempt(tableLabel, queueLabel); + } Duration delay = backoff.get(); logger.error("Failed to process batch of messages (retry in {})", delay, e); Thread.sleep(delay); @@ -166,7 +222,12 @@ public static void run( } while (!terminateAfterOneBatch); } - private static Collection filter(List messages, Collection matchers) { + private static Collection filter( + List messages, + Collection matchers, + InsertWatchMetrics metrics, + String tableLabel, + String queueLabel) { Collection r = new LinkedHashSet<>(); for (Message message : messages) { // Message body() example: @@ -193,23 +254,41 @@ private static Collection filter(List messages, Collection {}", eventName, target); // s3:ObjectCreated:{Put,Post,Copy,CompleteMultipartUpload} if (eventName.startsWith("ObjectCreated:")) { // TODO: exclude metadata/data dirs by default if (matchers.stream().anyMatch(matcher -> matcher.test(target))) { r.add(target); + if (metrics != null) { + metrics.recordEventMatched(tableLabel, queueLabel); + } + } else { + logger.info("Target did not match any input pattern: {}", target); + if (metrics != null) { + metrics.recordEventNotMatched(tableLabel, queueLabel); + } } } else { + if (metrics != null) { + metrics.recordEventSkipped(tableLabel, queueLabel); + } if (logger.isTraceEnabled()) { logger.trace("Message skipped: {} {}", eventName, target); } @@ -219,7 +298,13 @@ private static Collection filter(List messages, Collection messages) { + private static void confirmProcessed( + SqsClient sqs, + String sqsQueueURL, + List messages, + InsertWatchMetrics metrics, + String tableLabel, + String queueLabel) { int failedCount = 0; int len = messages.size(); for (int i = 0; i < len; i = i + 10) { @@ -228,6 +313,9 @@ private static void confirmProcessed(SqsClient sqs, String sqsQueueURL, List failed = res.failed(); failedCount += failed.size(); + if (metrics != null) { + metrics.recordSqsDeleteError(tableLabel, queueLabel, failed.size()); + } } } if (failedCount > 0) { @@ -252,4 +340,25 @@ private static DeleteMessageBatchResponse deleteMessageBatch( .toList()) .build()); } + + private static SqsClient buildSqsClient(String sqsQueueURL) { + SqsClientBuilder builder = SqsClient.builder(); + + // Extract endpoint from queue URL for non-AWS endpoints (e.g., ElasticMQ) + // AWS SQS URLs look like: https://sqs.us-east-1.amazonaws.com/123456789012/queue-name + // ElasticMQ URLs look like: http://localhost:9324/000000000000/queue-name + try { + URI uri = URI.create(sqsQueueURL); + String host = uri.getHost(); + if (host != null && !host.endsWith(".amazonaws.com")) { + URI endpoint = new URI(uri.getScheme(), null, host, uri.getPort(), null, null, null); + logger.info("Using custom SQS endpoint: {}", endpoint); + builder.endpointOverride(endpoint); + } + } catch (Exception e) { + logger.warn("Failed to parse SQS queue URL for endpoint extraction: {}", e.getMessage()); + } + + return builder.build(); + } } diff --git a/ice/src/main/java/com/altinity/ice/cli/internal/metrics/InsertWatchMetrics.java b/ice/src/main/java/com/altinity/ice/cli/internal/metrics/InsertWatchMetrics.java new file mode 100644 index 0000000..63f345f --- /dev/null +++ b/ice/src/main/java/com/altinity/ice/cli/internal/metrics/InsertWatchMetrics.java @@ -0,0 +1,263 @@ +/* + * Copyright (c) 2025 Altinity Inc and/or its affiliates. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.altinity.ice.cli.internal.metrics; + +import io.prometheus.metrics.core.metrics.Counter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Prometheus metrics for the InsertWatch (S3 watch) functionality. + * + *

This class uses a singleton pattern because Prometheus metrics can only be registered once per + * JVM. + */ +public class InsertWatchMetrics { + + private static final Logger logger = LoggerFactory.getLogger(InsertWatchMetrics.class); + + // Singleton instance + private static volatile InsertWatchMetrics instance; + private static final Object lock = new Object(); + + // ========================================================================== + // Metric Names + // ========================================================================== + + private static final String LABEL_TABLE = "table"; + private static final String LABEL_QUEUE = "queue"; + + private static final String[] WATCH_LABELS = {LABEL_TABLE, LABEL_QUEUE}; + + // Messages/Files processed + private static final String MESSAGES_RECEIVED_TOTAL_NAME = "ice_watch_messages_received_total"; + private static final String MESSAGES_RECEIVED_TOTAL_HELP = + "Total number of SQS messages received"; + + private static final String EVENTS_RECEIVED_TOTAL_NAME = "ice_watch_events_received_total"; + private static final String EVENTS_RECEIVED_TOTAL_HELP = + "Total number of S3 events received (one message may contain multiple events)"; + + private static final String EVENTS_MATCHED_TOTAL_NAME = "ice_watch_events_matched_total"; + private static final String EVENTS_MATCHED_TOTAL_HELP = + "Total number of S3 events that matched the pattern"; + + private static final String EVENTS_NOT_MATCHED_TOTAL_NAME = "ice_watch_events_not_matched_total"; + private static final String EVENTS_NOT_MATCHED_TOTAL_HELP = + "Total number of S3 events that did not match any input pattern"; + + private static final String EVENTS_SKIPPED_TOTAL_NAME = "ice_watch_events_skipped_total"; + private static final String EVENTS_SKIPPED_TOTAL_HELP = + "Total number of S3 events skipped (non-ObjectCreated events)"; + + // Files inserted + private static final String FILES_INSERTED_TOTAL_NAME = "ice_watch_files_inserted_total"; + private static final String FILES_INSERTED_TOTAL_HELP = + "Total number of files successfully inserted into the catalog"; + + // Transactions + private static final String TRANSACTIONS_TOTAL_NAME = "ice_watch_transactions_total"; + private static final String TRANSACTIONS_TOTAL_HELP = + "Total number of insert transactions committed"; + + private static final String TRANSACTIONS_FAILED_TOTAL_NAME = + "ice_watch_transactions_failed_total"; + private static final String TRANSACTIONS_FAILED_TOTAL_HELP = + "Total number of insert transactions that failed"; + + // Retry state + private static final String RETRY_ATTEMPTS_TOTAL_NAME = "ice_watch_retry_attempts_total"; + private static final String RETRY_ATTEMPTS_TOTAL_HELP = + "Total number of retry attempts due to failures"; + + // SQS errors + private static final String SQS_RECEIVE_ERRORS_TOTAL_NAME = "ice_watch_sqs_receive_errors_total"; + private static final String SQS_RECEIVE_ERRORS_TOTAL_HELP = + "Total number of errors when receiving messages from SQS"; + + private static final String SQS_DELETE_ERRORS_TOTAL_NAME = "ice_watch_sqs_delete_errors_total"; + private static final String SQS_DELETE_ERRORS_TOTAL_HELP = + "Total number of errors when deleting messages from SQS"; + + // Parse errors + private static final String MESSAGE_PARSE_ERRORS_TOTAL_NAME = + "ice_watch_message_parse_errors_total"; + private static final String MESSAGE_PARSE_ERRORS_TOTAL_HELP = + "Total number of message parsing errors"; + + // ========================================================================== + // Metrics + // ========================================================================== + + private final Counter messagesReceivedTotal; + private final Counter eventsReceivedTotal; + private final Counter eventsMatchedTotal; + private final Counter eventsNotMatchedTotal; + private final Counter eventsSkippedTotal; + private final Counter filesInsertedTotal; + private final Counter transactionsTotal; + private final Counter transactionsFailedTotal; + private final Counter retryAttemptsTotal; + private final Counter sqsReceiveErrorsTotal; + private final Counter sqsDeleteErrorsTotal; + private final Counter messageParseErrorsTotal; + + /** Returns the singleton instance of the metrics reporter. */ + public static InsertWatchMetrics getInstance() { + if (instance == null) { + synchronized (lock) { + if (instance == null) { + instance = new InsertWatchMetrics(); + } + } + } + return instance; + } + + private InsertWatchMetrics() { + this.messagesReceivedTotal = + Counter.builder() + .name(MESSAGES_RECEIVED_TOTAL_NAME) + .help(MESSAGES_RECEIVED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.eventsReceivedTotal = + Counter.builder() + .name(EVENTS_RECEIVED_TOTAL_NAME) + .help(EVENTS_RECEIVED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.eventsMatchedTotal = + Counter.builder() + .name(EVENTS_MATCHED_TOTAL_NAME) + .help(EVENTS_MATCHED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.eventsNotMatchedTotal = + Counter.builder() + .name(EVENTS_NOT_MATCHED_TOTAL_NAME) + .help(EVENTS_NOT_MATCHED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.eventsSkippedTotal = + Counter.builder() + .name(EVENTS_SKIPPED_TOTAL_NAME) + .help(EVENTS_SKIPPED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.filesInsertedTotal = + Counter.builder() + .name(FILES_INSERTED_TOTAL_NAME) + .help(FILES_INSERTED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.transactionsTotal = + Counter.builder() + .name(TRANSACTIONS_TOTAL_NAME) + .help(TRANSACTIONS_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.transactionsFailedTotal = + Counter.builder() + .name(TRANSACTIONS_FAILED_TOTAL_NAME) + .help(TRANSACTIONS_FAILED_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.retryAttemptsTotal = + Counter.builder() + .name(RETRY_ATTEMPTS_TOTAL_NAME) + .help(RETRY_ATTEMPTS_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.sqsReceiveErrorsTotal = + Counter.builder() + .name(SQS_RECEIVE_ERRORS_TOTAL_NAME) + .help(SQS_RECEIVE_ERRORS_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.sqsDeleteErrorsTotal = + Counter.builder() + .name(SQS_DELETE_ERRORS_TOTAL_NAME) + .help(SQS_DELETE_ERRORS_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + this.messageParseErrorsTotal = + Counter.builder() + .name(MESSAGE_PARSE_ERRORS_TOTAL_NAME) + .help(MESSAGE_PARSE_ERRORS_TOTAL_HELP) + .labelNames(WATCH_LABELS) + .register(); + + logger.info("InsertWatch Prometheus metrics initialized"); + } + + // ========================================================================== + // Public methods + // ========================================================================== + + public void recordMessagesReceived(String table, String queue, int count) { + messagesReceivedTotal.labelValues(table, queue).inc(count); + } + + public void recordEventsReceived(String table, String queue, int count) { + eventsReceivedTotal.labelValues(table, queue).inc(count); + } + + public void recordEventMatched(String table, String queue) { + eventsMatchedTotal.labelValues(table, queue).inc(); + } + + public void recordEventNotMatched(String table, String queue) { + eventsNotMatchedTotal.labelValues(table, queue).inc(); + } + + public void recordEventSkipped(String table, String queue) { + eventsSkippedTotal.labelValues(table, queue).inc(); + } + + public void recordFilesInserted(String table, String queue, int count) { + filesInsertedTotal.labelValues(table, queue).inc(count); + } + + public void recordTransactionSuccess(String table, String queue) { + transactionsTotal.labelValues(table, queue).inc(); + } + + public void recordTransactionFailed(String table, String queue) { + transactionsFailedTotal.labelValues(table, queue).inc(); + } + + public void recordRetryAttempt(String table, String queue) { + retryAttemptsTotal.labelValues(table, queue).inc(); + } + + public void recordSqsReceiveError(String table, String queue) { + sqsReceiveErrorsTotal.labelValues(table, queue).inc(); + } + + public void recordSqsDeleteError(String table, String queue, int count) { + sqsDeleteErrorsTotal.labelValues(table, queue).inc(count); + } + + public void recordMessageParseError(String table, String queue) { + messageParseErrorsTotal.labelValues(table, queue).inc(); + } +} From 18491d03ae2b318054f8c80c9e697417575318eb Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 9 Dec 2025 13:47:48 -0600 Subject: [PATCH 07/29] Added metrics for Maintenance. --- .../grafana/iceberg-metrics-dashboard.json | 931 ++++++++++++++++++ .../internal/maintenance/DataCompaction.java | 12 + .../maintenance/MaintenanceScheduler.java | 11 + .../internal/maintenance/OrphanCleanup.java | 19 + .../internal/metrics/MaintenanceMetrics.java | 263 +++++ 5 files changed, 1236 insertions(+) create mode 100644 ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index 8a378b1..44877ed 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -3042,6 +3042,937 @@ ], "title": "Errors Rate", "type": "timeseries" + }, + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 98 + }, + "id": 300, + "panels": [], + "title": "Maintenance", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [ + { + "options": { + "0": { + "color": "green", + "index": 0, + "text": "Idle" + }, + "1": { + "color": "orange", + "index": 1, + "text": "Running" + } + }, + "type": "value" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "orange", + "value": 1 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 0, + "y": 99 + }, + "id": 301, + "options": { + "colorMode": "background", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "ice_maintenance_in_progress or vector(0)", + "legendFormat": "Status", + "refId": "A" + } + ], + "title": "Maintenance Status", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 4, + "y": 99 + }, + "id": 302, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_maintenance_runs_total{status=\"success\"}[$__range])) or vector(0)", + "legendFormat": "Success", + "refId": "A" + } + ], + "title": "Successful Runs", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 1 + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 8, + "y": 99 + }, + "id": 303, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_maintenance_runs_total{status=\"failure\"}[$__range])) or vector(0)", + "legendFormat": "Failed", + "refId": "A" + } + ], + "title": "Failed Runs", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 12, + "y": 99 + }, + "id": 304, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_maintenance_orphan_files_deleted_total[$__range])) or vector(0)", + "legendFormat": "Deleted", + "refId": "A" + } + ], + "title": "Orphan Files Deleted", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 16, + "y": 99 + }, + "id": 305, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum(increase(ice_maintenance_compaction_files_merged_total[$__range])) or vector(0)", + "legendFormat": "Merged", + "refId": "A" + } + ], + "title": "Files Compacted", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "dateTimeFromNow" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 4, + "x": 20, + "y": 99 + }, + "id": 306, + "options": { + "colorMode": "value", + "graphMode": "none", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "max(ice_maintenance_last_run_timestamp) * 1000 or vector(0)", + "legendFormat": "Last Run", + "refId": "A" + } + ], + "title": "Last Maintenance Run", + "type": "stat" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 103 + }, + "id": 310, + "options": { + "legend": { + "calcs": ["mean", "max"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.50, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p50", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p95", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.99, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p99", + "refId": "C" + } + ], + "title": "Maintenance Duration", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*failure.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 103 + }, + "id": 311, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (status) (rate(ice_maintenance_runs_total[$__rate_interval]))", + "legendFormat": "{{status}}", + "refId": "A" + } + ], + "title": "Maintenance Run Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*Deleted.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "green", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": ".*Failed.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 111 + }, + "id": 312, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_orphan_files_found_total[$__rate_interval]))", + "legendFormat": "Found - {{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_orphan_files_deleted_total[$__rate_interval]))", + "legendFormat": "Deleted - {{table}}", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_orphan_delete_failures_total[$__rate_interval]))", + "legendFormat": "Failed - {{table}}", + "refId": "C" + } + ], + "title": "Orphan Cleanup Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 111 + }, + "id": 313, + "options": { + "legend": { + "calcs": ["sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_compaction_files_merged_total[$__rate_interval]))", + "legendFormat": "Merged - {{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_compaction_files_created_total[$__rate_interval]))", + "legendFormat": "Created - {{table}}", + "refId": "B" + } + ], + "title": "Compaction Files Rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + } + ] + }, + "unit": "binBps" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 119 + }, + "id": 314, + "options": { + "legend": { + "calcs": ["mean", "sum"], + "displayMode": "table", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "desc" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_compaction_bytes_read_total[$__rate_interval]))", + "legendFormat": "Read - {{table}}", + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_compaction_bytes_written_total[$__rate_interval]))", + "legendFormat": "Written - {{table}}", + "refId": "B" + } + ], + "title": "Compaction Throughput", + "type": "timeseries" } ], "refresh": "30s", diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java index 55e23fe..fe5de42 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java @@ -13,6 +13,7 @@ import com.altinity.ice.cli.internal.iceberg.RecordComparator; import com.altinity.ice.internal.iceberg.io.SchemeFileIO; import com.altinity.ice.internal.strings.Strings; +import com.altinity.ice.rest.catalog.internal.metrics.MaintenanceMetrics; import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; @@ -146,6 +147,8 @@ private void merge( FileIO tableIO = table.io(); Schema tableSchema = table.schema(); PartitionSpec tableSpec = table.spec(); + String tableName = table.name(); + MaintenanceMetrics maintenanceMetrics = MaintenanceMetrics.getInstance(); Transaction tx = table.newTransaction(); @@ -163,6 +166,10 @@ private void merge( return; } + // Calculate total bytes to read + long totalBytesToRead = dataFiles.stream().mapToLong(DataFile::fileSizeInBytes).sum(); + maintenanceMetrics.recordCompactionBytesRead(tableName, totalBytesToRead); + OutputFile outputFile = tableIO.newOutputFile(Strings.replacePrefix(dstDataFile, "s3://", "s3a://")); @@ -252,5 +259,10 @@ private void merge( delOp.commit(); tx.commitTransaction(); + + // Record metrics after successful commit + maintenanceMetrics.recordCompactionFilesMerged(tableName, dataFiles.size()); + maintenanceMetrics.recordCompactionFileCreated(tableName); + maintenanceMetrics.recordCompactionBytesWritten(tableName, dataFileSizeInBytes); } } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java index 2840415..c7210a8 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java @@ -9,6 +9,7 @@ */ package com.altinity.ice.rest.catalog.internal.maintenance; +import com.altinity.ice.rest.catalog.internal.metrics.MaintenanceMetrics; import com.github.shyiko.skedule.Schedule; import java.time.ZonedDateTime; import java.util.concurrent.ScheduledExecutorService; @@ -74,22 +75,32 @@ private void scheduleNextMaintenance() { } public void performMaintenance() { + MaintenanceMetrics metrics = MaintenanceMetrics.getInstance(); + if (isMaintenanceMode.get()) { logger.info("Skipping maintenance task as system is already in maintenance mode"); + metrics.recordMaintenanceSkipped(); return; } + long startTime = System.nanoTime(); + boolean success = false; + try { logger.info("Starting scheduled maintenance task"); setMaintenanceMode(true); + metrics.recordMaintenanceStarted(); maintenanceRunner.run(); logger.info("Scheduled maintenance task completed successfully"); + success = true; } catch (Exception e) { logger.error("Error during scheduled maintenance task", e); } finally { setMaintenanceMode(false); + double durationSecs = (System.nanoTime() - startTime) / 1_000_000_000.0; + metrics.recordMaintenanceCompleted(success, durationSecs); } } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/OrphanCleanup.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/OrphanCleanup.java index 94c28ab..e03f3b4 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/OrphanCleanup.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/OrphanCleanup.java @@ -11,6 +11,7 @@ import com.altinity.ice.internal.iceberg.io.SchemeFileIO; import com.altinity.ice.internal.io.Matcher; +import com.altinity.ice.rest.catalog.internal.metrics.MaintenanceMetrics; import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayDeque; @@ -21,6 +22,7 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; import org.apache.iceberg.ManifestFile; @@ -47,6 +49,8 @@ public record OrphanCleanup(long olderThanMillis, Matcher whitelist, boolean dry @Override public void perform(Table table) throws IOException { String location = table.location(); + String tableName = table.name(); + MaintenanceMetrics metrics = MaintenanceMetrics.getInstance(); logger.info("Searching for orphaned files at {}", location); @@ -64,6 +68,10 @@ public void perform(Table table) throws IOException { logger.info("Found {} orphaned file(s) ({} excluded)", orphanedFiles.size(), excluded); + // Record metrics + metrics.recordOrphanFilesFound(tableName, orphanedFiles.size() + excluded); + metrics.recordOrphanFilesExcluded(tableName, excluded); + if (orphanedFiles.isEmpty()) { return; } @@ -71,6 +79,9 @@ public void perform(Table table) throws IOException { if (!dryRun) { FileIO tableIO = table.io(); + AtomicInteger deletedCount = new AtomicInteger(0); + AtomicInteger failedCount = new AtomicInteger(0); + int numThreads = Math.min(8, orphanedFiles.size()); try (ExecutorService executor = Executors.newFixedThreadPool(numThreads)) { orphanedFiles.forEach( @@ -80,13 +91,21 @@ public void perform(Table table) throws IOException { try { logger.info("Deleting {}", file); tableIO.deleteFile(file); + deletedCount.incrementAndGet(); return file; } catch (Exception e) { logger.warn("Failed to delete file {}", file, e); + failedCount.incrementAndGet(); return null; } })); } + + // Record deletion metrics + metrics.recordOrphanFilesDeleted(tableName, deletedCount.get()); + for (int i = 0; i < failedCount.get(); i++) { + metrics.recordOrphanDeleteFailure(tableName); + } } else { orphanedFiles.stream().sorted().forEach(file -> logger.info("To be deleted: {}", file)); } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java new file mode 100644 index 0000000..53c889a --- /dev/null +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java @@ -0,0 +1,263 @@ +/* + * Copyright (c) 2025 Altinity Inc and/or its affiliates. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.altinity.ice.rest.catalog.internal.metrics; + +import io.prometheus.metrics.core.metrics.Counter; +import io.prometheus.metrics.core.metrics.Gauge; +import io.prometheus.metrics.core.metrics.Histogram; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MaintenanceMetrics { + + private static final Logger logger = LoggerFactory.getLogger(MaintenanceMetrics.class); + + private static volatile MaintenanceMetrics instance; + private static final Object lock = new Object(); + + // ========================================================================== + // Labels + // ========================================================================== + + private static final String LABEL_STATUS = "status"; + private static final String LABEL_TABLE = "table"; + + private static final String[] STATUS_LABELS = {LABEL_STATUS}; + + // ========================================================================== + // General Maintenance Metrics + // ========================================================================== + + private static final String RUNS_TOTAL_NAME = "ice_maintenance_runs_total"; + private static final String RUNS_TOTAL_HELP = "Total number of maintenance runs"; + + private static final String DURATION_SECONDS_NAME = "ice_maintenance_duration_seconds"; + private static final String DURATION_SECONDS_HELP = "Duration of maintenance run in seconds"; + + private static final String IN_PROGRESS_NAME = "ice_maintenance_in_progress"; + private static final String IN_PROGRESS_HELP = + "Whether maintenance is currently running (1 = running, 0 = idle)"; + + private static final String LAST_RUN_TIMESTAMP_NAME = "ice_maintenance_last_run_timestamp"; + private static final String LAST_RUN_TIMESTAMP_HELP = + "Unix timestamp of the last maintenance run"; + + private static final String SKIPPED_TOTAL_NAME = "ice_maintenance_skipped_total"; + private static final String SKIPPED_TOTAL_HELP = + "Times maintenance was skipped (already in maintenance mode)"; + + private static final String ORPHAN_FILES_FOUND_TOTAL_NAME = + "ice_maintenance_orphan_files_found_total"; + private static final String ORPHAN_FILES_FOUND_TOTAL_HELP = "Total orphaned files discovered"; + + private static final String ORPHAN_FILES_DELETED_TOTAL_NAME = + "ice_maintenance_orphan_files_deleted_total"; + private static final String ORPHAN_FILES_DELETED_TOTAL_HELP = + "Total orphaned files successfully deleted"; + + private static final String ORPHAN_FILES_EXCLUDED_TOTAL_NAME = + "ice_maintenance_orphan_files_excluded_total"; + private static final String ORPHAN_FILES_EXCLUDED_TOTAL_HELP = "Files excluded by whitelist"; + + private static final String ORPHAN_DELETE_FAILURES_TOTAL_NAME = + "ice_maintenance_orphan_delete_failures_total"; + private static final String ORPHAN_DELETE_FAILURES_TOTAL_HELP = "Files that failed to delete"; + + private static final String COMPACTION_FILES_MERGED_TOTAL_NAME = + "ice_maintenance_compaction_files_merged_total"; + private static final String COMPACTION_FILES_MERGED_TOTAL_HELP = "Total input files merged"; + + private static final String COMPACTION_FILES_CREATED_TOTAL_NAME = + "ice_maintenance_compaction_files_created_total"; + private static final String COMPACTION_FILES_CREATED_TOTAL_HELP = + "Total output files created after merge"; + + private static final String COMPACTION_BYTES_READ_TOTAL_NAME = + "ice_maintenance_compaction_bytes_read_total"; + private static final String COMPACTION_BYTES_READ_TOTAL_HELP = + "Total bytes read during compaction"; + + private static final String COMPACTION_BYTES_WRITTEN_TOTAL_NAME = + "ice_maintenance_compaction_bytes_written_total"; + private static final String COMPACTION_BYTES_WRITTEN_TOTAL_HELP = + "Total bytes written during compaction"; + + private static final double[] DURATION_BUCKETS = { + 0.1, 0.5, 1, 5, 10, 30, 60, 120, 300, 600, 1800, 3600 + }; + + // General + private final Counter runsTotal; + private final Histogram durationSeconds; + private final Gauge inProgress; + private final Gauge lastRunTimestamp; + private final Counter skippedTotal; + + // Orphan Cleanup + private final Counter orphanFilesFoundTotal; + private final Counter orphanFilesDeletedTotal; + private final Counter orphanFilesExcludedTotal; + private final Counter orphanDeleteFailuresTotal; + + // Data Compaction + private final Counter compactionFilesMergedTotal; + private final Counter compactionFilesCreatedTotal; + private final Counter compactionBytesReadTotal; + private final Counter compactionBytesWrittenTotal; + + /** Returns the singleton instance of the metrics. */ + public static MaintenanceMetrics getInstance() { + if (instance == null) { + synchronized (lock) { + if (instance == null) { + instance = new MaintenanceMetrics(); + } + } + } + return instance; + } + + private MaintenanceMetrics() { + // General maintenance metrics + this.runsTotal = + Counter.builder() + .name(RUNS_TOTAL_NAME) + .help(RUNS_TOTAL_HELP) + .labelNames(STATUS_LABELS) + .register(); + + this.durationSeconds = + Histogram.builder() + .name(DURATION_SECONDS_NAME) + .help(DURATION_SECONDS_HELP) + .classicUpperBounds(DURATION_BUCKETS) + .register(); + + this.inProgress = Gauge.builder().name(IN_PROGRESS_NAME).help(IN_PROGRESS_HELP).register(); + + this.lastRunTimestamp = + Gauge.builder() + .name(LAST_RUN_TIMESTAMP_NAME) + .help(LAST_RUN_TIMESTAMP_HELP) + .labelNames(STATUS_LABELS) + .register(); + + this.skippedTotal = + Counter.builder().name(SKIPPED_TOTAL_NAME).help(SKIPPED_TOTAL_HELP).register(); + + // Orphan cleanup metrics + this.orphanFilesFoundTotal = + Counter.builder() + .name(ORPHAN_FILES_FOUND_TOTAL_NAME) + .help(ORPHAN_FILES_FOUND_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + this.orphanFilesDeletedTotal = + Counter.builder() + .name(ORPHAN_FILES_DELETED_TOTAL_NAME) + .help(ORPHAN_FILES_DELETED_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + this.orphanFilesExcludedTotal = + Counter.builder() + .name(ORPHAN_FILES_EXCLUDED_TOTAL_NAME) + .help(ORPHAN_FILES_EXCLUDED_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + this.orphanDeleteFailuresTotal = + Counter.builder() + .name(ORPHAN_DELETE_FAILURES_TOTAL_NAME) + .help(ORPHAN_DELETE_FAILURES_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + // Data compaction metrics + this.compactionFilesMergedTotal = + Counter.builder() + .name(COMPACTION_FILES_MERGED_TOTAL_NAME) + .help(COMPACTION_FILES_MERGED_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + this.compactionFilesCreatedTotal = + Counter.builder() + .name(COMPACTION_FILES_CREATED_TOTAL_NAME) + .help(COMPACTION_FILES_CREATED_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + this.compactionBytesReadTotal = + Counter.builder() + .name(COMPACTION_BYTES_READ_TOTAL_NAME) + .help(COMPACTION_BYTES_READ_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + this.compactionBytesWrittenTotal = + Counter.builder() + .name(COMPACTION_BYTES_WRITTEN_TOTAL_NAME) + .help(COMPACTION_BYTES_WRITTEN_TOTAL_HELP) + .labelNames(LABEL_TABLE) + .register(); + + logger.info("Maintenance Prometheus metrics initialized"); + } + + public void recordMaintenanceStarted() { + inProgress.set(1.0); + } + + public void recordMaintenanceCompleted(boolean success, double durationSecs) { + String status = success ? "success" : "failure"; + runsTotal.labelValues(status).inc(); + durationSeconds.observe(durationSecs); + lastRunTimestamp.labelValues(status).set(System.currentTimeMillis() / 1000.0); + inProgress.set(0.0); + } + + public void recordMaintenanceSkipped() { + skippedTotal.inc(); + } + + public void recordOrphanFilesFound(String table, int count) { + orphanFilesFoundTotal.labelValues(table).inc(count); + } + + public void recordOrphanFilesDeleted(String table, int count) { + orphanFilesDeletedTotal.labelValues(table).inc(count); + } + + public void recordOrphanFilesExcluded(String table, int count) { + orphanFilesExcludedTotal.labelValues(table).inc(count); + } + + public void recordOrphanDeleteFailure(String table) { + orphanDeleteFailuresTotal.labelValues(table).inc(); + } + + public void recordCompactionFilesMerged(String table, int count) { + compactionFilesMergedTotal.labelValues(table).inc(count); + } + + public void recordCompactionFileCreated(String table) { + compactionFilesCreatedTotal.labelValues(table).inc(); + } + + public void recordCompactionBytesRead(String table, long bytes) { + compactionBytesReadTotal.labelValues(table).inc(bytes); + } + + public void recordCompactionBytesWritten(String table, long bytes) { + compactionBytesWrittenTotal.labelValues(table).inc(bytes); + } +} From 3ce34665d985aab1e5c239b7f60c444de69a8968 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Thu, 11 Dec 2025 12:38:45 -0600 Subject: [PATCH 08/29] Renamed metrics as it collides with prometheus reserved names. --- .../grafana/iceberg-metrics-dashboard.json | 2 +- .../internal/maintenance/DataCompaction.java | 2 +- .../maintenance/MaintenanceScheduler.java | 22 +++++++++++-------- .../internal/metrics/MaintenanceMetrics.java | 20 ++++++++--------- 4 files changed, 25 insertions(+), 21 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index 44877ed..e4ffb12 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -3868,7 +3868,7 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_files_created_total[$__rate_interval]))", + "expr": "sum by (table) (rate(ice_maintenance_compaction_output_files_total[$__rate_interval]))", "legendFormat": "Created - {{table}}", "refId": "B" } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java index fe5de42..df7c4c4 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/DataCompaction.java @@ -262,7 +262,7 @@ private void merge( // Record metrics after successful commit maintenanceMetrics.recordCompactionFilesMerged(tableName, dataFiles.size()); - maintenanceMetrics.recordCompactionFileCreated(tableName); + maintenanceMetrics.recordCompactionOutputFile(tableName); maintenanceMetrics.recordCompactionBytesWritten(tableName, dataFileSizeInBytes); } } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java index c7210a8..107c772 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java @@ -75,18 +75,20 @@ private void scheduleNextMaintenance() { } public void performMaintenance() { - MaintenanceMetrics metrics = MaintenanceMetrics.getInstance(); - - if (isMaintenanceMode.get()) { - logger.info("Skipping maintenance task as system is already in maintenance mode"); - metrics.recordMaintenanceSkipped(); - return; - } long startTime = System.nanoTime(); boolean success = false; - + MaintenanceMetrics metrics = null; try { + + metrics = MaintenanceMetrics.getInstance(); + + if (isMaintenanceMode.get()) { + logger.info("Skipping maintenance task as system is already in maintenance mode"); + metrics.recordMaintenanceSkipped(); + return; + } + logger.info("Starting scheduled maintenance task"); setMaintenanceMode(true); metrics.recordMaintenanceStarted(); @@ -100,7 +102,9 @@ public void performMaintenance() { } finally { setMaintenanceMode(false); double durationSecs = (System.nanoTime() - startTime) / 1_000_000_000.0; - metrics.recordMaintenanceCompleted(success, durationSecs); + if (metrics != null) { + metrics.recordMaintenanceCompleted(success, durationSecs); + } } } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java index 53c889a..7e3688e 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java @@ -74,10 +74,10 @@ public class MaintenanceMetrics { "ice_maintenance_compaction_files_merged_total"; private static final String COMPACTION_FILES_MERGED_TOTAL_HELP = "Total input files merged"; - private static final String COMPACTION_FILES_CREATED_TOTAL_NAME = - "ice_maintenance_compaction_files_created_total"; - private static final String COMPACTION_FILES_CREATED_TOTAL_HELP = - "Total output files created after merge"; + private static final String COMPACTION_OUTPUT_FILES_TOTAL_NAME = + "ice_maintenance_compaction_output_files_total"; + private static final String COMPACTION_OUTPUT_FILES_TOTAL_HELP = + "Total output files produced after merge"; private static final String COMPACTION_BYTES_READ_TOTAL_NAME = "ice_maintenance_compaction_bytes_read_total"; @@ -108,7 +108,7 @@ public class MaintenanceMetrics { // Data Compaction private final Counter compactionFilesMergedTotal; - private final Counter compactionFilesCreatedTotal; + private final Counter compactionOutputFilesTotal; private final Counter compactionBytesReadTotal; private final Counter compactionBytesWrittenTotal; @@ -189,10 +189,10 @@ private MaintenanceMetrics() { .labelNames(LABEL_TABLE) .register(); - this.compactionFilesCreatedTotal = + this.compactionOutputFilesTotal = Counter.builder() - .name(COMPACTION_FILES_CREATED_TOTAL_NAME) - .help(COMPACTION_FILES_CREATED_TOTAL_HELP) + .name(COMPACTION_OUTPUT_FILES_TOTAL_NAME) + .help(COMPACTION_OUTPUT_FILES_TOTAL_HELP) .labelNames(LABEL_TABLE) .register(); @@ -249,8 +249,8 @@ public void recordCompactionFilesMerged(String table, int count) { compactionFilesMergedTotal.labelValues(table).inc(count); } - public void recordCompactionFileCreated(String table) { - compactionFilesCreatedTotal.labelValues(table).inc(); + public void recordCompactionOutputFile(String table) { + compactionOutputFilesTotal.labelValues(table).inc(); } public void recordCompactionBytesRead(String table, long bytes) { From 5d0b79f5b12f3faa3043e2b6bee932924b739102 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Mon, 15 Dec 2025 17:59:25 -0600 Subject: [PATCH 09/29] Update dashboard for metrics. --- .../grafana/iceberg-metrics-dashboard.json | 86 ++++++++++++++++++- .../internal/metrics/MaintenanceMetrics.java | 10 +++ 2 files changed, 95 insertions(+), 1 deletion(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index e4ffb12..43a728a 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -3202,6 +3202,7 @@ "color": { "mode": "palette-classic" }, + "decimals": 0, "mappings": [], "thresholds": { "mode": "absolute", @@ -3428,6 +3429,89 @@ "title": "Last Maintenance Run", "type": "stat" }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "description": "Shows how long maintenance has been running. Use this for alerting if maintenance is stuck (e.g., alert if > 1 hour).", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "mappings": [ + { + "options": { + "match": "null", + "result": { + "color": "green", + "index": 0, + "text": "Not Running" + } + }, + "type": "special" + } + ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "yellow", + "value": 300 + }, + { + "color": "orange", + "value": 1800 + }, + { + "color": "red", + "value": 3600 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 8, + "x": 0, + "y": 103 + }, + "id": 307, + "options": { + "colorMode": "background", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": ["lastNotNull"], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.0.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "(time() - ice_maintenance_start_timestamp) * (ice_maintenance_in_progress == 1) * (ice_maintenance_start_timestamp > 0)", + "legendFormat": "Duration", + "refId": "A" + } + ], + "title": "Current Maintenance Duration", + "type": "stat" + }, { "datasource": { "type": "prometheus", @@ -3488,7 +3572,7 @@ "h": 8, "w": 12, "x": 0, - "y": 103 + "y": 107 }, "id": 310, "options": { diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java index 7e3688e..59cadcb 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/MaintenanceMetrics.java @@ -49,6 +49,10 @@ public class MaintenanceMetrics { private static final String LAST_RUN_TIMESTAMP_HELP = "Unix timestamp of the last maintenance run"; + private static final String START_TIMESTAMP_NAME = "ice_maintenance_start_timestamp"; + private static final String START_TIMESTAMP_HELP = + "Unix timestamp when current maintenance started (0 if not running)"; + private static final String SKIPPED_TOTAL_NAME = "ice_maintenance_skipped_total"; private static final String SKIPPED_TOTAL_HELP = "Times maintenance was skipped (already in maintenance mode)"; @@ -98,6 +102,7 @@ public class MaintenanceMetrics { private final Histogram durationSeconds; private final Gauge inProgress; private final Gauge lastRunTimestamp; + private final Gauge startTimestamp; private final Counter skippedTotal; // Orphan Cleanup @@ -149,6 +154,9 @@ private MaintenanceMetrics() { .labelNames(STATUS_LABELS) .register(); + this.startTimestamp = + Gauge.builder().name(START_TIMESTAMP_NAME).help(START_TIMESTAMP_HELP).register(); + this.skippedTotal = Counter.builder().name(SKIPPED_TOTAL_NAME).help(SKIPPED_TOTAL_HELP).register(); @@ -215,6 +223,7 @@ private MaintenanceMetrics() { public void recordMaintenanceStarted() { inProgress.set(1.0); + startTimestamp.set(System.currentTimeMillis() / 1000.0); } public void recordMaintenanceCompleted(boolean success, double durationSecs) { @@ -223,6 +232,7 @@ public void recordMaintenanceCompleted(boolean success, double durationSecs) { durationSeconds.observe(durationSecs); lastRunTimestamp.labelValues(status).set(System.currentTimeMillis() / 1000.0); inProgress.set(0.0); + startTimestamp.set(0.0); } public void recordMaintenanceSkipped() { From 99d8d3db4505a2ce4916b4e5d83040d4232c7dfc Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Mon, 15 Dec 2025 18:01:33 -0600 Subject: [PATCH 10/29] Update dashboard for metrics. --- .../grafana/iceberg-metrics-dashboard.json | 689 +++++++++--------- 1 file changed, 350 insertions(+), 339 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index 43a728a..c779767 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -18,8 +18,9 @@ "editable": true, "fiscalYearStartMonth": 0, "graphTooltip": 1, - "id": null, + "id": 2, "links": [], + "liveNow": false, "panels": [ { "collapsed": false, @@ -71,21 +72,25 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum(increase(iceberg_scans_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "editorMode": "code", + "expr": "sum(increase(iceberg_scans_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range]))", "legendFormat": "Scans", + "range": true, "refId": "A" } ], @@ -95,7 +100,7 @@ { "datasource": { "type": "prometheus", - "uid": "${datasource}" + "uid": "PBFA97CFB590B2093" }, "fieldConfig": { "defaults": { @@ -129,21 +134,25 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", - "uid": "${datasource}" + "uid": "PBFA97CFB590B2093" }, - "expr": "sum(increase(iceberg_commits_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "editorMode": "code", + "expr": "sum(iceberg_commits_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"})", "legendFormat": "Commits", + "range": true, "refId": "A" } ], @@ -191,13 +200,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -215,65 +226,7 @@ { "datasource": { "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "bytes" - }, - "overrides": [] - }, - "gridPos": { - "h": 4, - "w": 4, - "x": 12, - "y": 1 - }, - "id": 4, - "options": { - "colorMode": "value", - "graphMode": "area", - "justifyMode": "auto", - "orientation": "auto", - "reduceOptions": { - "calcs": ["lastNotNull"], - "fields": "", - "values": false - }, - "textMode": "auto" - }, - "pluginVersion": "10.0.0", - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum(increase(iceberg_scan_total_file_size_bytes{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", - "legendFormat": "Bytes Scanned", - "refId": "A" - } - ], - "title": "Data Scanned", - "type": "stat" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" + "uid": "PBFA97CFB590B2093" }, "fieldConfig": { "defaults": { @@ -307,105 +260,31 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", - "uid": "${datasource}" + "uid": "PBFA97CFB590B2093" }, - "expr": "sum(increase(iceberg_commit_added_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) or vector(0)", + "editorMode": "code", + "expr": "sum(iceberg_commit_added_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"})", "legendFormat": "Records Added", + "range": true, "refId": "A" } ], "title": "Records Added", "type": "stat" }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "mappings": [ - { - "options": { - "0": { - "color": "red", - "index": 1, - "text": "Inactive" - }, - "1": { - "color": "green", - "index": 0, - "text": "Active" - } - }, - "type": "value" - } - ], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "red", - "value": null - }, - { - "color": "green", - "value": 1 - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 4, - "w": 4, - "x": 20, - "y": 1 - }, - "id": 6, - "options": { - "colorMode": "value", - "graphMode": "none", - "justifyMode": "auto", - "orientation": "auto", - "reduceOptions": { - "calcs": ["lastNotNull"], - "fields": "", - "values": false - }, - "textMode": "auto" - }, - "pluginVersion": "10.0.0", - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "iceberg_metrics_reporter_active or vector(0)", - "legendFormat": "Reporter Status", - "refId": "A" - } - ], - "title": "Metrics Reporter", - "type": "stat" - }, { "datasource": { "type": "prometheus", @@ -471,7 +350,9 @@ "id": 7, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -560,7 +441,9 @@ "id": 8, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -662,7 +545,10 @@ "id": 10, "options": { "legend": { - "calcs": ["mean", "max"], + "calcs": [ + "mean", + "max" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -769,7 +655,10 @@ "id": 11, "options": { "legend": { - "calcs": ["mean", "max"], + "calcs": [ + "mean", + "max" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -867,7 +756,9 @@ "id": 12, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -883,8 +774,10 @@ "type": "prometheus", "uid": "${datasource}" }, + "editorMode": "code", "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_scanned_data_manifests{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", "legendFormat": "Scanned - {{catalog}}.{{namespace}}.{{table}}", + "range": true, "refId": "A" }, { @@ -965,7 +858,10 @@ "id": 13, "options": { "legend": { - "calcs": ["mean", "sum"], + "calcs": [ + "mean", + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1067,7 +963,10 @@ "id": 20, "options": { "legend": { - "calcs": ["mean", "max"], + "calcs": [ + "mean", + "max" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1174,7 +1073,9 @@ "id": 21, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1272,7 +1173,9 @@ "id": 22, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1370,7 +1273,10 @@ "id": 23, "options": { "legend": { - "calcs": ["mean", "sum"], + "calcs": [ + "mean", + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1472,7 +1378,9 @@ "id": 30, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1570,7 +1478,9 @@ "id": 31, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -1653,13 +1563,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -1719,13 +1631,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -1777,13 +1691,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -1843,13 +1759,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -1929,7 +1847,9 @@ "id": 44, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -2049,7 +1969,9 @@ "id": 45, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -2138,7 +2060,10 @@ "id": 46, "options": { "legend": { - "calcs": ["mean", "max"], + "calcs": [ + "mean", + "max" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -2245,7 +2170,9 @@ "id": 47, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -2319,13 +2246,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -2377,13 +2306,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -2435,13 +2366,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -2497,13 +2430,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -2559,13 +2494,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -2645,7 +2582,9 @@ "id": 210, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -2752,7 +2691,9 @@ "id": 211, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -2857,7 +2798,9 @@ "id": 212, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -3001,7 +2944,9 @@ "id": 213, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -3113,21 +3058,25 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", "uid": "${datasource}" }, - "expr": "ice_maintenance_in_progress or vector(0)", + "editorMode": "code", + "expr": "ice_maintenance_in_progress", "legendFormat": "Status", + "range": true, "refId": "A" } ], @@ -3171,13 +3120,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -3234,13 +3185,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -3292,13 +3245,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -3350,13 +3305,15 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { @@ -3408,21 +3365,25 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", "uid": "${datasource}" }, - "expr": "max(ice_maintenance_last_run_timestamp) * 1000 or vector(0)", + "editorMode": "code", + "expr": "max(ice_maintenance_last_run_timestamp) * 1000", "legendFormat": "Last Run", + "range": true, "refId": "A" } ], @@ -3491,21 +3452,25 @@ "justifyMode": "auto", "orientation": "auto", "reduceOptions": { - "calcs": ["lastNotNull"], + "calcs": [ + "lastNotNull" + ], "fields": "", "values": false }, "textMode": "auto" }, - "pluginVersion": "10.0.0", + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", "uid": "${datasource}" }, - "expr": "(time() - ice_maintenance_start_timestamp) * (ice_maintenance_in_progress == 1) * (ice_maintenance_start_timestamp > 0)", + "editorMode": "code", + "expr": "(time() - ice_maintenance_start_timestamp)", "legendFormat": "Duration", + "range": true, "refId": "A" } ], @@ -3564,20 +3529,38 @@ } ] }, - "unit": "s" + "unit": "short" }, - "overrides": [] + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*failure.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] }, "gridPos": { "h": 8, "w": 12, - "x": 0, - "y": 107 + "x": 12, + "y": 103 }, - "id": 310, + "id": 311, "options": { "legend": { - "calcs": ["mean", "max"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -3593,30 +3576,12 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "histogram_quantile(0.50, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", - "legendFormat": "p50", + "expr": "sum by (status) (rate(ice_maintenance_runs_total[$__rate_interval]))", + "legendFormat": "{{status}}", "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "histogram_quantile(0.95, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", - "legendFormat": "p95", - "refId": "B" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "histogram_quantile(0.99, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", - "legendFormat": "p99", - "refId": "C" } ], - "title": "Maintenance Duration", + "title": "Maintenance Run Rate", "type": "timeseries" }, { @@ -3671,36 +3636,23 @@ } ] }, - "unit": "short" + "unit": "s" }, - "overrides": [ - { - "matcher": { - "id": "byRegexp", - "options": ".*failure.*" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "red", - "mode": "fixed" - } - } - ] - } - ] + "overrides": [] }, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 103 + "x": 0, + "y": 107 }, - "id": 311, + "id": 310, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "mean", + "max" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -3716,12 +3668,30 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum by (status) (rate(ice_maintenance_runs_total[$__rate_interval]))", - "legendFormat": "{{status}}", + "expr": "histogram_quantile(0.50, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p50", "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p95", + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.99, sum by (le) (rate(ice_maintenance_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p99", + "refId": "C" } ], - "title": "Maintenance Run Rate", + "title": "Maintenance Duration", "type": "timeseries" }, { @@ -3778,49 +3748,20 @@ }, "unit": "short" }, - "overrides": [ - { - "matcher": { - "id": "byRegexp", - "options": ".*Deleted.*" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "green", - "mode": "fixed" - } - } - ] - }, - { - "matcher": { - "id": "byRegexp", - "options": ".*Failed.*" - }, - "properties": [ - { - "id": "color", - "value": { - "fixedColor": "red", - "mode": "fixed" - } - } - ] - } - ] + "overrides": [] }, "gridPos": { "h": 8, "w": 12, - "x": 0, + "x": 12, "y": 111 }, - "id": 312, + "id": 313, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -3836,8 +3777,8 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum by (table) (rate(ice_maintenance_orphan_files_found_total[$__rate_interval]))", - "legendFormat": "Found - {{table}}", + "expr": "sum by (table) (rate(ice_maintenance_compaction_files_merged_total[$__rate_interval]))", + "legendFormat": "Merged - {{table}}", "refId": "A" }, { @@ -3845,21 +3786,12 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum by (table) (rate(ice_maintenance_orphan_files_deleted_total[$__rate_interval]))", - "legendFormat": "Deleted - {{table}}", + "expr": "sum by (table) (rate(ice_maintenance_compaction_output_files_total[$__rate_interval]))", + "legendFormat": "Created - {{table}}", "refId": "B" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (table) (rate(ice_maintenance_orphan_delete_failures_total[$__rate_interval]))", - "legendFormat": "Failed - {{table}}", - "refId": "C" } ], - "title": "Orphan Cleanup Rate", + "title": "Compaction Files Rate", "type": "timeseries" }, { @@ -3916,18 +3848,51 @@ }, "unit": "short" }, - "overrides": [] + "overrides": [ + { + "matcher": { + "id": "byRegexp", + "options": ".*Deleted.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "green", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byRegexp", + "options": ".*Failed.*" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "red", + "mode": "fixed" + } + } + ] + } + ] }, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 111 + "x": 0, + "y": 115 }, - "id": 313, + "id": 312, "options": { "legend": { - "calcs": ["sum"], + "calcs": [ + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -3943,8 +3908,8 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_files_merged_total[$__rate_interval]))", - "legendFormat": "Merged - {{table}}", + "expr": "sum by (table) (rate(ice_maintenance_orphan_files_found_total[$__rate_interval]))", + "legendFormat": "Found - {{table}}", "refId": "A" }, { @@ -3952,12 +3917,21 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_output_files_total[$__rate_interval]))", - "legendFormat": "Created - {{table}}", + "expr": "sum by (table) (rate(ice_maintenance_orphan_files_deleted_total[$__rate_interval]))", + "legendFormat": "Deleted - {{table}}", "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "sum by (table) (rate(ice_maintenance_orphan_delete_failures_total[$__rate_interval]))", + "legendFormat": "Failed - {{table}}", + "refId": "C" } ], - "title": "Compaction Files Rate", + "title": "Orphan Cleanup Rate", "type": "timeseries" }, { @@ -4020,12 +3994,15 @@ "h": 8, "w": 24, "x": 0, - "y": 119 + "y": 123 }, "id": 314, "options": { "legend": { - "calcs": ["mean", "sum"], + "calcs": [ + "mean", + "sum" + ], "displayMode": "table", "placement": "bottom", "showLegend": true @@ -4059,13 +4036,22 @@ "type": "timeseries" } ], - "refresh": "30s", + "refresh": "auto", "schemaVersion": 38, - "tags": ["iceberg", "data-lake", "rest-catalog", "s3-watch"], + "tags": [ + "iceberg", + "data-lake", + "rest-catalog", + "s3-watch" + ], "templating": { "list": [ { - "current": {}, + "current": { + "selected": false, + "text": "Prometheus", + "value": "PBFA97CFB590B2093" + }, "hide": 0, "includeAll": false, "label": "Datasource", @@ -4083,8 +4069,12 @@ "allValue": ".*", "current": { "selected": true, - "text": "All", - "value": "$__all" + "text": [ + "All" + ], + "value": [ + "$__all" + ] }, "datasource": { "type": "prometheus", @@ -4111,8 +4101,14 @@ "allValue": ".*", "current": { "selected": true, - "text": "All", - "value": "$__all" + "text": [ + "default.flowers", + "default.nyc" + ], + "value": [ + "default.flowers", + "default.nyc" + ] }, "datasource": { "type": "prometheus", @@ -4139,8 +4135,12 @@ "allValue": ".*", "current": { "selected": true, - "text": "All", - "value": "$__all" + "text": [ + "All" + ], + "value": [ + "$__all" + ] }, "datasource": { "type": "prometheus", @@ -4167,8 +4167,12 @@ "allValue": ".*", "current": { "selected": true, - "text": "All", - "value": "$__all" + "text": [ + "All" + ], + "value": [ + "$__all" + ] }, "datasource": { "type": "prometheus", @@ -4195,8 +4199,12 @@ "allValue": ".*", "current": { "selected": true, - "text": "All", - "value": "$__all" + "text": [ + "All" + ], + "value": [ + "$__all" + ] }, "datasource": { "type": "prometheus", @@ -4223,8 +4231,12 @@ "allValue": ".*", "current": { "selected": true, - "text": "All", - "value": "$__all" + "text": [ + "All" + ], + "value": [ + "$__all" + ] }, "datasource": { "type": "prometheus", @@ -4250,14 +4262,13 @@ ] }, "time": { - "from": "now-1h", + "from": "now-90d", "to": "now" }, "timepicker": {}, "timezone": "browser", "title": "Iceberg REST Catalog Metrics", "uid": "iceberg-rest-catalog-metrics", - "version": 1, + "version": 27, "weekStart": "" -} - +} \ No newline at end of file From a1edefa9353ccff602f73a3d37aa7cd27d406a8e Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Mon, 15 Dec 2025 18:45:15 -0600 Subject: [PATCH 11/29] Update dashboard for metrics. --- .../grafana/iceberg-metrics-dashboard.json | 282 +++--------------- 1 file changed, 44 insertions(+), 238 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index c779767..b464cf3 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -250,7 +250,7 @@ "gridPos": { "h": 4, "w": 4, - "x": 16, + "x": 12, "y": 1 }, "id": 5, @@ -691,207 +691,13 @@ "title": "Data Files Per Scan", "type": "timeseries" }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "normal" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 22 - }, - "id": 12, - "options": { - "legend": { - "calcs": [ - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "editorMode": "code", - "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_scanned_data_manifests{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Scanned - {{catalog}}.{{namespace}}.{{table}}", - "range": true, - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_skipped_data_manifests{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Skipped - {{catalog}}.{{namespace}}.{{table}}", - "refId": "B" - } - ], - "title": "Manifests Scanned vs Skipped", - "type": "timeseries" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "binBps" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 22 - }, - "id": 13, - "options": { - "legend": { - "calcs": [ - "mean", - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_total_file_size_bytes{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "{{catalog}}.{{namespace}}.{{table}}", - "refId": "A" - } - ], - "title": "Scan Throughput (bytes/sec)", - "type": "timeseries" - }, { "collapsed": false, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 30 + "y": 22 }, "id": 102, "panels": [], @@ -958,7 +764,7 @@ "h": 8, "w": 12, "x": 0, - "y": 31 + "y": 23 }, "id": 20, "options": { @@ -1068,7 +874,7 @@ "h": 8, "w": 12, "x": 12, - "y": 31 + "y": 23 }, "id": 21, "options": { @@ -1168,7 +974,7 @@ "h": 8, "w": 12, "x": 0, - "y": 39 + "y": 31 }, "id": 22, "options": { @@ -1268,7 +1074,7 @@ "h": 8, "w": 12, "x": 12, - "y": 39 + "y": 31 }, "id": 23, "options": { @@ -1306,7 +1112,7 @@ "h": 1, "w": 24, "x": 0, - "y": 47 + "y": 39 }, "id": 103, "panels": [], @@ -1373,7 +1179,7 @@ "h": 8, "w": 12, "x": 0, - "y": 48 + "y": 40 }, "id": 30, "options": { @@ -1473,7 +1279,7 @@ "h": 8, "w": 12, "x": 12, - "y": 48 + "y": 40 }, "id": 31, "options": { @@ -1519,7 +1325,7 @@ "h": 1, "w": 24, "x": 0, - "y": 56 + "y": 48 }, "id": 104, "panels": [], @@ -1554,7 +1360,7 @@ "h": 4, "w": 6, "x": 0, - "y": 57 + "y": 49 }, "id": 40, "options": { @@ -1622,7 +1428,7 @@ "h": 4, "w": 6, "x": 6, - "y": 57 + "y": 49 }, "id": 41, "options": { @@ -1682,7 +1488,7 @@ "h": 4, "w": 6, "x": 12, - "y": 57 + "y": 49 }, "id": 42, "options": { @@ -1750,7 +1556,7 @@ "h": 4, "w": 6, "x": 18, - "y": 57 + "y": 49 }, "id": 43, "options": { @@ -1842,7 +1648,7 @@ "h": 8, "w": 12, "x": 0, - "y": 61 + "y": 53 }, "id": 44, "options": { @@ -1964,7 +1770,7 @@ "h": 8, "w": 12, "x": 12, - "y": 61 + "y": 53 }, "id": 45, "options": { @@ -2055,7 +1861,7 @@ "h": 8, "w": 12, "x": 0, - "y": 69 + "y": 61 }, "id": 46, "options": { @@ -2165,7 +1971,7 @@ "h": 8, "w": 12, "x": 12, - "y": 69 + "y": 61 }, "id": 47, "options": { @@ -2202,7 +2008,7 @@ "h": 1, "w": 24, "x": 0, - "y": 77 + "y": 69 }, "id": 200, "panels": [], @@ -2237,7 +2043,7 @@ "h": 4, "w": 5, "x": 0, - "y": 78 + "y": 70 }, "id": 202, "options": { @@ -2297,7 +2103,7 @@ "h": 4, "w": 5, "x": 5, - "y": 78 + "y": 70 }, "id": 203, "options": { @@ -2357,7 +2163,7 @@ "h": 4, "w": 5, "x": 10, - "y": 78 + "y": 70 }, "id": 204, "options": { @@ -2421,7 +2227,7 @@ "h": 4, "w": 5, "x": 15, - "y": 78 + "y": 70 }, "id": 205, "options": { @@ -2485,7 +2291,7 @@ "h": 4, "w": 4, "x": 20, - "y": 78 + "y": 70 }, "id": 206, "options": { @@ -2577,7 +2383,7 @@ "h": 8, "w": 12, "x": 0, - "y": 82 + "y": 74 }, "id": 210, "options": { @@ -2686,7 +2492,7 @@ "h": 8, "w": 12, "x": 12, - "y": 82 + "y": 74 }, "id": 211, "options": { @@ -2793,7 +2599,7 @@ "h": 8, "w": 12, "x": 0, - "y": 90 + "y": 82 }, "id": 212, "options": { @@ -2939,7 +2745,7 @@ "h": 8, "w": 12, "x": 12, - "y": 90 + "y": 82 }, "id": 213, "options": { @@ -2994,7 +2800,7 @@ "h": 1, "w": 24, "x": 0, - "y": 98 + "y": 90 }, "id": 300, "panels": [], @@ -3049,7 +2855,7 @@ "h": 4, "w": 4, "x": 0, - "y": 99 + "y": 91 }, "id": 301, "options": { @@ -3111,7 +2917,7 @@ "h": 4, "w": 4, "x": 4, - "y": 99 + "y": 91 }, "id": 302, "options": { @@ -3176,7 +2982,7 @@ "h": 4, "w": 4, "x": 8, - "y": 99 + "y": 91 }, "id": 303, "options": { @@ -3236,7 +3042,7 @@ "h": 4, "w": 4, "x": 12, - "y": 99 + "y": 91 }, "id": 304, "options": { @@ -3296,7 +3102,7 @@ "h": 4, "w": 4, "x": 16, - "y": 99 + "y": 91 }, "id": 305, "options": { @@ -3356,7 +3162,7 @@ "h": 4, "w": 4, "x": 20, - "y": 99 + "y": 91 }, "id": 306, "options": { @@ -3443,7 +3249,7 @@ "h": 4, "w": 8, "x": 0, - "y": 103 + "y": 95 }, "id": 307, "options": { @@ -3553,7 +3359,7 @@ "h": 8, "w": 12, "x": 12, - "y": 103 + "y": 95 }, "id": 311, "options": { @@ -3644,7 +3450,7 @@ "h": 8, "w": 12, "x": 0, - "y": 107 + "y": 99 }, "id": 310, "options": { @@ -3754,7 +3560,7 @@ "h": 8, "w": 12, "x": 12, - "y": 111 + "y": 103 }, "id": 313, "options": { @@ -3885,7 +3691,7 @@ "h": 8, "w": 12, "x": 0, - "y": 115 + "y": 107 }, "id": 312, "options": { @@ -3994,7 +3800,7 @@ "h": 8, "w": 24, "x": 0, - "y": 123 + "y": 115 }, "id": 314, "options": { @@ -4036,7 +3842,7 @@ "type": "timeseries" } ], - "refresh": "auto", + "refresh": "5s", "schemaVersion": 38, "tags": [ "iceberg", @@ -4269,6 +4075,6 @@ "timezone": "browser", "title": "Iceberg REST Catalog Metrics", "uid": "iceberg-rest-catalog-metrics", - "version": 27, + "version": 31, "weekStart": "" -} \ No newline at end of file +} From 694a633f1f31d5a4d6606e6a86f119c7e9aad2aa Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 16 Dec 2025 12:16:28 -0600 Subject: [PATCH 12/29] Update dashboard for metrics. --- .../grafana/iceberg-metrics-dashboard.json | 266 +++--------------- examples/scratch/README.md | 5 +- 2 files changed, 42 insertions(+), 229 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index b464cf3..e9ecf03 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -788,8 +788,10 @@ "type": "prometheus", "uid": "${datasource}" }, + "editorMode": "code", "expr": "histogram_quantile(0.50, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", "legendFormat": "p50 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "range": true, "refId": "A" }, { @@ -914,205 +916,13 @@ "title": "Data Files Added/Removed", "type": "timeseries" }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 31 - }, - "id": 22, - "options": { - "legend": { - "calcs": [ - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (catalog, namespace, table) (rate(iceberg_commit_added_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Added - {{catalog}}.{{namespace}}.{{table}}", - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "-sum by (catalog, namespace, table) (rate(iceberg_commit_removed_records_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Removed - {{catalog}}.{{namespace}}.{{table}}", - "refId": "B" - } - ], - "title": "Records Added/Removed Rate", - "type": "timeseries" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "binBps" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 31 - }, - "id": 23, - "options": { - "legend": { - "calcs": [ - "mean", - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (catalog, namespace, table) (rate(iceberg_commit_total_files_size_bytes{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "{{catalog}}.{{namespace}}.{{table}}", - "refId": "A" - } - ], - "title": "Commit Throughput (bytes/sec)", - "type": "timeseries" - }, { "collapsed": false, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 39 + "y": 31 }, "id": 103, "panels": [], @@ -1179,7 +989,7 @@ "h": 8, "w": 12, "x": 0, - "y": 40 + "y": 32 }, "id": 30, "options": { @@ -1279,7 +1089,7 @@ "h": 8, "w": 12, "x": 12, - "y": 40 + "y": 32 }, "id": 31, "options": { @@ -1325,7 +1135,7 @@ "h": 1, "w": 24, "x": 0, - "y": 48 + "y": 40 }, "id": 104, "panels": [], @@ -1360,7 +1170,7 @@ "h": 4, "w": 6, "x": 0, - "y": 49 + "y": 41 }, "id": 40, "options": { @@ -1428,7 +1238,7 @@ "h": 4, "w": 6, "x": 6, - "y": 49 + "y": 41 }, "id": 41, "options": { @@ -1488,7 +1298,7 @@ "h": 4, "w": 6, "x": 12, - "y": 49 + "y": 41 }, "id": 42, "options": { @@ -1556,7 +1366,7 @@ "h": 4, "w": 6, "x": 18, - "y": 49 + "y": 41 }, "id": 43, "options": { @@ -1648,7 +1458,7 @@ "h": 8, "w": 12, "x": 0, - "y": 53 + "y": 45 }, "id": 44, "options": { @@ -1770,7 +1580,7 @@ "h": 8, "w": 12, "x": 12, - "y": 53 + "y": 45 }, "id": 45, "options": { @@ -1861,7 +1671,7 @@ "h": 8, "w": 12, "x": 0, - "y": 61 + "y": 53 }, "id": 46, "options": { @@ -1971,7 +1781,7 @@ "h": 8, "w": 12, "x": 12, - "y": 61 + "y": 53 }, "id": 47, "options": { @@ -2008,7 +1818,7 @@ "h": 1, "w": 24, "x": 0, - "y": 69 + "y": 61 }, "id": 200, "panels": [], @@ -2043,7 +1853,7 @@ "h": 4, "w": 5, "x": 0, - "y": 70 + "y": 62 }, "id": 202, "options": { @@ -2103,7 +1913,7 @@ "h": 4, "w": 5, "x": 5, - "y": 70 + "y": 62 }, "id": 203, "options": { @@ -2163,7 +1973,7 @@ "h": 4, "w": 5, "x": 10, - "y": 70 + "y": 62 }, "id": 204, "options": { @@ -2227,7 +2037,7 @@ "h": 4, "w": 5, "x": 15, - "y": 70 + "y": 62 }, "id": 205, "options": { @@ -2291,7 +2101,7 @@ "h": 4, "w": 4, "x": 20, - "y": 70 + "y": 62 }, "id": 206, "options": { @@ -2383,7 +2193,7 @@ "h": 8, "w": 12, "x": 0, - "y": 74 + "y": 66 }, "id": 210, "options": { @@ -2492,7 +2302,7 @@ "h": 8, "w": 12, "x": 12, - "y": 74 + "y": 66 }, "id": 211, "options": { @@ -2599,7 +2409,7 @@ "h": 8, "w": 12, "x": 0, - "y": 82 + "y": 74 }, "id": 212, "options": { @@ -2745,7 +2555,7 @@ "h": 8, "w": 12, "x": 12, - "y": 82 + "y": 74 }, "id": 213, "options": { @@ -2800,7 +2610,7 @@ "h": 1, "w": 24, "x": 0, - "y": 90 + "y": 82 }, "id": 300, "panels": [], @@ -2855,7 +2665,7 @@ "h": 4, "w": 4, "x": 0, - "y": 91 + "y": 83 }, "id": 301, "options": { @@ -2917,7 +2727,7 @@ "h": 4, "w": 4, "x": 4, - "y": 91 + "y": 83 }, "id": 302, "options": { @@ -2982,7 +2792,7 @@ "h": 4, "w": 4, "x": 8, - "y": 91 + "y": 83 }, "id": 303, "options": { @@ -3042,7 +2852,7 @@ "h": 4, "w": 4, "x": 12, - "y": 91 + "y": 83 }, "id": 304, "options": { @@ -3102,7 +2912,7 @@ "h": 4, "w": 4, "x": 16, - "y": 91 + "y": 83 }, "id": 305, "options": { @@ -3162,7 +2972,7 @@ "h": 4, "w": 4, "x": 20, - "y": 91 + "y": 83 }, "id": 306, "options": { @@ -3249,7 +3059,7 @@ "h": 4, "w": 8, "x": 0, - "y": 95 + "y": 87 }, "id": 307, "options": { @@ -3359,7 +3169,7 @@ "h": 8, "w": 12, "x": 12, - "y": 95 + "y": 87 }, "id": 311, "options": { @@ -3450,7 +3260,7 @@ "h": 8, "w": 12, "x": 0, - "y": 99 + "y": 91 }, "id": 310, "options": { @@ -3560,7 +3370,7 @@ "h": 8, "w": 12, "x": 12, - "y": 103 + "y": 95 }, "id": 313, "options": { @@ -3691,7 +3501,7 @@ "h": 8, "w": 12, "x": 0, - "y": 107 + "y": 99 }, "id": 312, "options": { @@ -3800,7 +3610,7 @@ "h": 8, "w": 24, "x": 0, - "y": 115 + "y": 107 }, "id": 314, "options": { @@ -4077,4 +3887,4 @@ "uid": "iceberg-rest-catalog-metrics", "version": 31, "weekStart": "" -} +} \ No newline at end of file diff --git a/examples/scratch/README.md b/examples/scratch/README.md index e2e50bb..5948549 100644 --- a/examples/scratch/README.md +++ b/examples/scratch/README.md @@ -34,7 +34,10 @@ ice insert nyc.taxis -p \ ice insert nyc.taxis_p_by_day -p \ https://d37ci6vzurychx.cloudfront.net/trip-data/yellow_tripdata_2025-01.parquet \ --partition='[{"column":"tpep_pickup_datetime","transform":"day"}]' - + +# delete partition +ice delete nyc.taxis_p_by_day --partition '[{"name": "tpep_pickup_datetime", "values": ["2024-12-31T23:51:20"]}]' --dry-run=false + # insert data ordered by tpep_pickup_datetime column ice insert nyc.taxis_s_by_day -p \ https://d37ci6vzurychx.cloudfront.net/trip-data/yellow_tripdata_2025-01.parquet \ From db27ed9cb833f97271b66b4c2f9b4a92024089a5 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 16 Dec 2025 12:23:29 -0600 Subject: [PATCH 13/29] Update dashboard for metrics. --- .../grafana/iceberg-metrics-dashboard.json | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index e9ecf03..04d6523 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -819,7 +819,7 @@ { "datasource": { "type": "prometheus", - "uid": "${datasource}" + "uid": "PBFA97CFB590B2093" }, "fieldConfig": { "defaults": { @@ -834,7 +834,7 @@ "axisPlacement": "auto", "barAlignment": 0, "drawStyle": "line", - "fillOpacity": 10, + "fillOpacity": 0, "gradientMode": "none", "hideFrom": { "legend": false, @@ -848,7 +848,7 @@ "scaleDistribution": { "type": "linear" }, - "showPoints": "never", + "showPoints": "auto", "spanNulls": false, "stacking": { "group": "A", @@ -881,36 +881,30 @@ "id": 21, "options": { "legend": { - "calcs": [ - "sum" - ], - "displayMode": "table", + "calcs": [], + "displayMode": "list", "placement": "bottom", "showLegend": true }, "tooltip": { - "mode": "multi", - "sort": "desc" + "mode": "single", + "sort": "none" } }, + "pluginVersion": "10.2.0", "targets": [ { "datasource": { "type": "prometheus", - "uid": "${datasource}" + "uid": "PBFA97CFB590B2093" }, - "expr": "sum by (catalog, namespace, table, operation) (rate(iceberg_commit_added_data_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Added - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", + "editorMode": "code", + "expr": "sum by (catalog, namespace, table, operation) (iceberg_commit_added_data_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"})", + "hide": false, + "instant": false, + "legendFormat": "__auto", + "range": true, "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "-sum by (catalog, namespace, table, operation) (rate(iceberg_commit_removed_data_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Removed - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", - "refId": "B" } ], "title": "Data Files Added/Removed", From 8bcae99ec450b82b9c4a48d575d59fab52e18001 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 16 Dec 2025 12:59:42 -0600 Subject: [PATCH 14/29] Update dashboard for metrics. --- .../grafana/iceberg-metrics-dashboard.json | 231 +----------------- 1 file changed, 8 insertions(+), 223 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index 04d6523..14657bf 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -789,28 +789,10 @@ "uid": "${datasource}" }, "editorMode": "code", - "expr": "histogram_quantile(0.50, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", + "expr": "sum by (catalog, namespace, table, operation) (increase(iceberg_commit_duration_seconds_sum{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__range])) ", "legendFormat": "p50 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", "range": true, "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "histogram_quantile(0.95, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", - "legendFormat": "p95 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", - "refId": "B" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "histogram_quantile(0.99, sum by (le, catalog, namespace, table, operation) (rate(iceberg_commit_duration_seconds_bucket{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval])))", - "legendFormat": "p99 - {{catalog}}.{{namespace}}.{{table}} ({{operation}})", - "refId": "C" } ], "title": "Commit Duration", @@ -1006,8 +988,10 @@ "type": "prometheus", "uid": "${datasource}" }, + "editorMode": "code", "expr": "sum by (catalog, namespace, table) (rate(iceberg_scan_positional_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", "legendFormat": "Positional - {{catalog}}.{{namespace}}.{{table}}", + "range": true, "refId": "A" }, { @@ -1023,106 +1007,6 @@ "title": "Delete Files Scanned by Type", "type": "timeseries" }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 32 - }, - "id": 31, - "options": { - "legend": { - "calcs": [ - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (catalog, namespace, table) (rate(iceberg_commit_added_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Added - {{catalog}}.{{namespace}}.{{table}}", - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "-sum by (catalog, namespace, table) (rate(iceberg_commit_removed_delete_files_total{catalog=~\"$catalog\", namespace=~\"$namespace\", table=~\"$table\"}[$__rate_interval]))", - "legendFormat": "Removed - {{catalog}}.{{namespace}}.{{table}}", - "refId": "B" - } - ], - "title": "Delete Files Added/Removed", - "type": "timeseries" - }, { "collapsed": false, "gridPos": { @@ -2319,8 +2203,10 @@ "type": "prometheus", "uid": "${datasource}" }, + "editorMode": "code", "expr": "sum by (table, queue) (rate(ice_watch_files_inserted_total{table=~\"$watch_table\", queue=~\"$watch_queue\", instance=~\"$watch_instance\"}[$__rate_interval]))", "legendFormat": "{{table}}", + "range": true, "refId": "A" } ], @@ -3543,107 +3429,6 @@ ], "title": "Orphan Cleanup Rate", "type": "timeseries" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "binBps" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 24, - "x": 0, - "y": 107 - }, - "id": 314, - "options": { - "legend": { - "calcs": [ - "mean", - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_bytes_read_total[$__rate_interval]))", - "legendFormat": "Read - {{table}}", - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_bytes_written_total[$__rate_interval]))", - "legendFormat": "Written - {{table}}", - "refId": "B" - } - ], - "title": "Compaction Throughput", - "type": "timeseries" } ], "refresh": "5s", @@ -3746,10 +3531,10 @@ "current": { "selected": true, "text": [ - "All" + "iris" ], "value": [ - "$__all" + "iris" ] }, "datasource": { @@ -3881,4 +3666,4 @@ "uid": "iceberg-rest-catalog-metrics", "version": 31, "weekStart": "" -} \ No newline at end of file +} From 31532b9f50b69c2990e06fde0de3dbb429c641c0 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 16 Dec 2025 15:48:29 -0600 Subject: [PATCH 15/29] Added logic to track response size --- .../catalog/internal/metrics/HttpMetrics.java | 22 +++++++++++++++++-- .../internal/metrics/IcebergMetricNames.java | 3 +++ .../internal/rest/RESTCatalogServlet.java | 12 +++++++--- ice/src/main/resources/logback.xml | 2 ++ 4 files changed, 34 insertions(+), 5 deletions(-) diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java index ae060e9..778532c 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java @@ -25,6 +25,7 @@ public class HttpMetrics { private final Counter responsesTotal; private final Histogram requestDuration; private final Gauge requestsInFlight; + private final Counter responseSizeBytes; public static HttpMetrics getInstance() { if (instance == null) { @@ -65,6 +66,13 @@ private HttpMetrics() { .name(HTTP_REQUESTS_IN_FLIGHT_NAME) .help(HTTP_REQUESTS_IN_FLIGHT_HELP) .register(); + + this.responseSizeBytes = + Counter.builder() + .name(HTTP_RESPONSE_SIZE_BYTES_NAME) + .help(HTTP_RESPONSE_SIZE_BYTES_HELP) + .labelNames(HTTP_REQUEST_LABELS) + .register(); } public void recordRequestStart(String method, String route) { @@ -72,13 +80,17 @@ public void recordRequestStart(String method, String route) { requestsInFlight.inc(); } - public void recordRequestEnd(String method, String route, int statusCode, long startTimeNanos) { + public void recordRequestEnd( + String method, String route, int statusCode, long startTimeNanos, long responseSize) { requestsInFlight.dec(); double durationSeconds = (System.nanoTime() - startTimeNanos) / (double) TimeUnit.SECONDS.toNanos(1); requestDuration.labelValues(method, route).observe(durationSeconds); responsesTotal.labelValues(method, route, Integer.toString(statusCode)).inc(); + if (responseSize > 0) { + responseSizeBytes.labelValues(method, route).inc(responseSize); + } } public RequestTimer startRequest(String method, String route) { @@ -91,6 +103,7 @@ public static class RequestTimer implements AutoCloseable { private final String route; private final long startTimeNanos; private int statusCode = 200; + private long responseSize = 0; RequestTimer(HttpMetrics metrics, String method, String route) { this.metrics = metrics; @@ -105,9 +118,14 @@ public void setStatusCode(int statusCode) { this.statusCode = statusCode; } + /** Set the response size in bytes before closing. */ + public void setResponseSize(long responseSize) { + this.responseSize = responseSize; + } + @Override public void close() { - metrics.recordRequestEnd(method, route, statusCode, startTimeNanos); + metrics.recordRequestEnd(method, route, statusCode, startTimeNanos, responseSize); } } } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java index 997e02a..43150f3 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/IcebergMetricNames.java @@ -188,6 +188,9 @@ private IcebergMetricNames() {} public static final String HTTP_REQUESTS_IN_FLIGHT_HELP = "Number of HTTP requests currently being processed"; + public static final String HTTP_RESPONSE_SIZE_BYTES_NAME = "iceberg_http_response_size_bytes"; + public static final String HTTP_RESPONSE_SIZE_BYTES_HELP = "HTTP response size in bytes"; + /** HTTP request duration buckets (in seconds) - suitable for REST API calls. */ public static final double[] HTTP_DURATION_BUCKETS = { 0.001, 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1, 2.5, 5, 10 diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java index afb227d..4b2563e 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/rest/RESTCatalogServlet.java @@ -97,7 +97,9 @@ protected void handle(HttpServletRequest request, HttpServletResponse response) .withType("BadRequestException") .withMessage(String.format("No route for %s %s", method, path)) .build(); - RESTObjectMapper.mapper().writeValue(response.getWriter(), res); + byte[] responseBytes = RESTObjectMapper.mapper().writeValueAsBytes(res); + timer.setResponseSize(responseBytes.length); + response.getOutputStream().write(responseBytes); } return; } @@ -152,7 +154,9 @@ protected void handle(HttpServletRequest request, HttpServletResponse response) timer.setStatusCode(error.code()); response.setStatus(error.code()); response.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); - RESTObjectMapper.mapper().writeValue(response.getWriter(), error); + byte[] errorBytes = RESTObjectMapper.mapper().writeValueAsBytes(error); + timer.setResponseSize(errorBytes.length); + response.getOutputStream().write(errorBytes); return; } @@ -160,7 +164,9 @@ protected void handle(HttpServletRequest request, HttpServletResponse response) response.setStatus(HttpServletResponse.SC_OK); response.setHeader(HttpHeaders.CONTENT_TYPE, ContentType.APPLICATION_JSON.getMimeType()); if (responseBody != null) { - RESTObjectMapper.mapper().writeValue(response.getWriter(), responseBody); + byte[] responseBytes = RESTObjectMapper.mapper().writeValueAsBytes(responseBody); + timer.setResponseSize(responseBytes.length); + response.getOutputStream().write(responseBytes); } } } diff --git a/ice/src/main/resources/logback.xml b/ice/src/main/resources/logback.xml index d731bd8..73f2b17 100644 --- a/ice/src/main/resources/logback.xml +++ b/ice/src/main/resources/logback.xml @@ -14,6 +14,8 @@ + + From a0c67488ad0e87109fba2634ef58b4adebaa1692 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 16 Dec 2025 22:01:37 -0600 Subject: [PATCH 16/29] Updated grafana dashboard. --- .../grafana/iceberg-metrics-dashboard.json | 344 ++++++------------ .../catalog/internal/metrics/HttpMetrics.java | 3 + 2 files changed, 109 insertions(+), 238 deletions(-) diff --git a/examples/grafana/iceberg-metrics-dashboard.json b/examples/grafana/iceberg-metrics-dashboard.json index 14657bf..9644502 100644 --- a/examples/grafana/iceberg-metrics-dashboard.json +++ b/examples/grafana/iceberg-metrics-dashboard.json @@ -1225,18 +1225,10 @@ { "color": "green", "value": null - }, - { - "color": "yellow", - "value": 0.1 - }, - { - "color": "red", - "value": 0.5 } ] }, - "unit": "s" + "unit": "bytes" }, "overrides": [] }, @@ -1246,7 +1238,7 @@ "x": 18, "y": 41 }, - "id": 43, + "id": 60, "options": { "colorMode": "value", "graphMode": "area", @@ -1268,12 +1260,12 @@ "type": "prometheus", "uid": "${datasource}" }, - "expr": "histogram_quantile(0.95, sum by (le) (rate(iceberg_http_request_duration_seconds_bucket[$__rate_interval])))", - "legendFormat": "p95 Latency", + "expr": "sum(iceberg_http_response_size_bytes_total)", + "legendFormat": "Total Bytes", "refId": "A" } ], - "title": "p95 Latency", + "title": "Response Size", "type": "stat" }, { @@ -1367,6 +1359,74 @@ "title": "Request Rate", "type": "timeseries" }, + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "yellow", + "value": 0.1 + }, + { + "color": "red", + "value": 0.5 + } + ] + }, + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 4, + "w": 6, + "x": 18, + "y": 45 + }, + "id": 43, + "options": { + "colorMode": "value", + "graphMode": "area", + "justifyMode": "auto", + "orientation": "auto", + "reduceOptions": { + "calcs": [ + "lastNotNull" + ], + "fields": "", + "values": false + }, + "textMode": "auto" + }, + "pluginVersion": "10.2.0", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${datasource}" + }, + "expr": "histogram_quantile(0.95, sum by (le) (rate(iceberg_http_request_duration_seconds_bucket[$__rate_interval])))", + "legendFormat": "p95 Latency", + "refId": "A" + } + ], + "title": "p95 Latency", + "type": "stat" + }, { "datasource": { "type": "prometheus", @@ -1458,7 +1518,7 @@ "h": 8, "w": 12, "x": 12, - "y": 45 + "y": 49 }, "id": 45, "options": { @@ -1659,7 +1719,7 @@ "h": 8, "w": 12, "x": 12, - "y": 53 + "y": 57 }, "id": 47, "options": { @@ -1696,7 +1756,7 @@ "h": 1, "w": 24, "x": 0, - "y": 61 + "y": 65 }, "id": 200, "panels": [], @@ -1731,7 +1791,7 @@ "h": 4, "w": 5, "x": 0, - "y": 62 + "y": 66 }, "id": 202, "options": { @@ -1791,7 +1851,7 @@ "h": 4, "w": 5, "x": 5, - "y": 62 + "y": 66 }, "id": 203, "options": { @@ -1851,7 +1911,7 @@ "h": 4, "w": 5, "x": 10, - "y": 62 + "y": 66 }, "id": 204, "options": { @@ -1915,7 +1975,7 @@ "h": 4, "w": 5, "x": 15, - "y": 62 + "y": 66 }, "id": 205, "options": { @@ -1979,7 +2039,7 @@ "h": 4, "w": 4, "x": 20, - "y": 62 + "y": 66 }, "id": 206, "options": { @@ -2071,7 +2131,7 @@ "h": 8, "w": 12, "x": 0, - "y": 66 + "y": 70 }, "id": 210, "options": { @@ -2180,7 +2240,7 @@ "h": 8, "w": 12, "x": 12, - "y": 66 + "y": 70 }, "id": 211, "options": { @@ -2289,7 +2349,7 @@ "h": 8, "w": 12, "x": 0, - "y": 74 + "y": 78 }, "id": 212, "options": { @@ -2435,7 +2495,7 @@ "h": 8, "w": 12, "x": 12, - "y": 74 + "y": 78 }, "id": 213, "options": { @@ -2490,95 +2550,13 @@ "h": 1, "w": 24, "x": 0, - "y": 82 + "y": 86 }, "id": 300, "panels": [], "title": "Maintenance", "type": "row" }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "mappings": [ - { - "options": { - "0": { - "color": "green", - "index": 0, - "text": "Idle" - }, - "1": { - "color": "orange", - "index": 1, - "text": "Running" - } - }, - "type": "value" - } - ], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "orange", - "value": 1 - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 4, - "w": 4, - "x": 0, - "y": 83 - }, - "id": 301, - "options": { - "colorMode": "background", - "graphMode": "none", - "justifyMode": "auto", - "orientation": "auto", - "reduceOptions": { - "calcs": [ - "lastNotNull" - ], - "fields": "", - "values": false - }, - "textMode": "auto" - }, - "pluginVersion": "10.2.0", - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "editorMode": "code", - "expr": "ice_maintenance_in_progress", - "legendFormat": "Status", - "range": true, - "refId": "A" - } - ], - "title": "Maintenance Status", - "type": "stat" - }, { "datasource": { "type": "prometheus", @@ -2606,8 +2584,8 @@ "gridPos": { "h": 4, "w": 4, - "x": 4, - "y": 83 + "x": 0, + "y": 87 }, "id": 302, "options": { @@ -2671,8 +2649,8 @@ "gridPos": { "h": 4, "w": 4, - "x": 8, - "y": 83 + "x": 4, + "y": 87 }, "id": 303, "options": { @@ -2732,7 +2710,7 @@ "h": 4, "w": 4, "x": 12, - "y": 83 + "y": 87 }, "id": 304, "options": { @@ -2792,7 +2770,7 @@ "h": 4, "w": 4, "x": 16, - "y": 83 + "y": 87 }, "id": 305, "options": { @@ -2852,7 +2830,7 @@ "h": 4, "w": 4, "x": 20, - "y": 83 + "y": 87 }, "id": 306, "options": { @@ -2939,7 +2917,7 @@ "h": 4, "w": 8, "x": 0, - "y": 87 + "y": 91 }, "id": 307, "options": { @@ -3049,7 +3027,7 @@ "h": 8, "w": 12, "x": 12, - "y": 87 + "y": 91 }, "id": 311, "options": { @@ -3140,7 +3118,7 @@ "h": 8, "w": 12, "x": 0, - "y": 91 + "y": 95 }, "id": 310, "options": { @@ -3190,106 +3168,6 @@ "title": "Maintenance Duration", "type": "timeseries" }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 95 - }, - "id": 313, - "options": { - "legend": { - "calcs": [ - "sum" - ], - "displayMode": "table", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "desc" - } - }, - "targets": [ - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_files_merged_total[$__rate_interval]))", - "legendFormat": "Merged - {{table}}", - "refId": "A" - }, - { - "datasource": { - "type": "prometheus", - "uid": "${datasource}" - }, - "expr": "sum by (table) (rate(ice_maintenance_compaction_output_files_total[$__rate_interval]))", - "legendFormat": "Created - {{table}}", - "refId": "B" - } - ], - "title": "Compaction Files Rate", - "type": "timeseries" - }, { "datasource": { "type": "prometheus", @@ -3381,7 +3259,7 @@ "h": 8, "w": 12, "x": 0, - "y": 99 + "y": 103 }, "id": 312, "options": { @@ -3431,7 +3309,7 @@ "type": "timeseries" } ], - "refresh": "5s", + "refresh": "", "schemaVersion": 38, "tags": [ "iceberg", @@ -3495,15 +3373,9 @@ { "allValue": ".*", "current": { - "selected": true, - "text": [ - "default.flowers", - "default.nyc" - ], - "value": [ - "default.flowers", - "default.nyc" - ] + "selected": false, + "text": "All", + "value": "$__all" }, "datasource": { "type": "prometheus", @@ -3529,13 +3401,9 @@ { "allValue": ".*", "current": { - "selected": true, - "text": [ - "iris" - ], - "value": [ - "iris" - ] + "selected": false, + "text": "All", + "value": "$__all" }, "datasource": { "type": "prometheus", @@ -3657,13 +3525,13 @@ ] }, "time": { - "from": "now-90d", + "from": "now-24h", "to": "now" }, "timepicker": {}, "timezone": "browser", "title": "Iceberg REST Catalog Metrics", "uid": "iceberg-rest-catalog-metrics", - "version": 31, + "version": 40, "weekStart": "" } diff --git a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java index 778532c..5a17d52 100644 --- a/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java +++ b/ice-rest-catalog/src/main/java/com/altinity/ice/rest/catalog/internal/metrics/HttpMetrics.java @@ -73,6 +73,9 @@ private HttpMetrics() { .help(HTTP_RESPONSE_SIZE_BYTES_HELP) .labelNames(HTTP_REQUEST_LABELS) .register(); + + // Initialize with zero to make metrics visible immediately + this.responseSizeBytes.labelValues("GET", "CONFIG").inc(0); } public void recordRequestStart(String method, String route) { From dd2e19752cc322314b7363220ee84b23bddc9460 Mon Sep 17 00:00:00 2001 From: kanthi subramanian Date: Tue, 16 Dec 2025 22:20:50 -0600 Subject: [PATCH 17/29] Added documentation for grafana dashboard. --- examples/grafana/METRICS.md | 135 ++++++++++++++++++ .../screenshots/commit-delete-metrics.png | Bin 0 -> 239661 bytes examples/grafana/screenshots/http-metrics.png | Bin 0 -> 311140 bytes .../screenshots/insertwatch-metrics.png | Bin 0 -> 215636 bytes .../screenshots/maintenance-metrics.png | Bin 0 -> 390013 bytes .../screenshots/overview-scan-metrics.png | Bin 0 -> 319771 bytes 6 files changed, 135 insertions(+) create mode 100644 examples/grafana/METRICS.md create mode 100644 examples/grafana/screenshots/commit-delete-metrics.png create mode 100644 examples/grafana/screenshots/http-metrics.png create mode 100644 examples/grafana/screenshots/insertwatch-metrics.png create mode 100644 examples/grafana/screenshots/maintenance-metrics.png create mode 100644 examples/grafana/screenshots/overview-scan-metrics.png diff --git a/examples/grafana/METRICS.md b/examples/grafana/METRICS.md new file mode 100644 index 0000000..e9746b3 --- /dev/null +++ b/examples/grafana/METRICS.md @@ -0,0 +1,135 @@ +# Iceberg REST Catalog Metrics + +This document describes the Prometheus metrics exposed by the Iceberg REST Catalog. + +## Dashboard Overview + +### Overview & Scan Metrics +![Overview and Scan Metrics](screenshots/overview-scan-metrics.png) + +### Commit Metrics & Delete Files +![Commit Metrics and Delete Files](screenshots/commit-delete-metrics.png) + +### HTTP/REST Endpoint Metrics +![HTTP REST Endpoint Metrics](screenshots/http-metrics.png) + +### InsertWatch (S3 Watch) Metrics +![InsertWatch S3 Watch Metrics](screenshots/insertwatch-metrics.png) + +### Maintenance Metrics +![Maintenance Metrics](screenshots/maintenance-metrics.png) + +## Metrics Reference + +### Iceberg Table Metrics + +These metrics are reported by Iceberg clients when they perform operations on tables. + +#### Scan Metrics + +| Metric Name | Type | Labels | Description | +|-------------|------|--------|-------------| +| `iceberg_scans_total` | Counter | catalog, namespace, table | Total number of Iceberg table scans | +| `iceberg_scan_result_data_files_total` | Counter | catalog, namespace, table | Total number of data files in scan results | +| `iceberg_scan_result_delete_files_total` | Counter | catalog, namespace, table | Total number of delete files in scan results | +| `iceberg_scan_indexed_delete_files_total` | Counter | catalog, namespace, table | Total number of indexed delete files in scan results | +| `iceberg_scan_positional_delete_files_total` | Counter | catalog, namespace, table | Total number of positional delete files in scan results | +| `iceberg_scan_equality_delete_files_total` | Counter | catalog, namespace, table | Total number of equality delete files in scan results | +| `iceberg_scan_total_data_manifests` | Counter | catalog, namespace, table | Total number of data manifests considered during scans | +| `iceberg_scan_total_delete_manifests` | Counter | catalog, namespace, table | Total number of delete manifests considered during scans | +| `iceberg_scan_scanned_data_manifests` | Counter | catalog, namespace, table | Total number of data manifests actually scanned | +| `iceberg_scan_skipped_data_manifests` | Counter | catalog, namespace, table | Total number of data manifests skipped during scans | +| `iceberg_scan_total_file_size_bytes` | Counter | catalog, namespace, table | Total file size in bytes for scanned data files | +| `iceberg_scan_total_delete_file_size_bytes` | Counter | catalog, namespace, table | Total file size in bytes for scanned delete files | +| `iceberg_scan_planning_duration_seconds` | Histogram | catalog, namespace, table | Duration of scan planning in seconds | +| `iceberg_scan_data_files_per_scan` | Histogram | catalog, namespace, table | Distribution of data files per scan | + +#### Commit Metrics + +| Metric Name | Type | Labels | Description | +|-------------|------|--------|-------------| +| `iceberg_commits_total` | Counter | catalog, namespace, table, operation | Total number of Iceberg table commits | +| `iceberg_commit_added_data_files_total` | Counter | catalog, namespace, table, operation | Total number of data files added in commits | +| `iceberg_commit_removed_data_files_total` | Counter | catalog, namespace, table, operation | Total number of data files removed in commits | +| `iceberg_commit_added_delete_files_total` | Counter | catalog, namespace, table, operation | Total number of delete files added in commits | +| `iceberg_commit_removed_delete_files_total` | Counter | catalog, namespace, table, operation | Total number of delete files removed in commits | +| `iceberg_commit_added_records_total` | Counter | catalog, namespace, table, operation | Total number of records added in commits | +| `iceberg_commit_removed_records_total` | Counter | catalog, namespace, table, operation | Total number of records removed in commits | +| `iceberg_commit_added_equality_deletes_total` | Counter | catalog, namespace, table, operation | Total number of equality deletes added in commits | +| `iceberg_commit_total_files_size_bytes` | Counter | catalog, namespace, table, operation | Total size in bytes of files involved in commits | +| `iceberg_commit_duration_seconds` | Histogram | catalog, namespace, table, operation | Duration of commit operations in seconds | + +#### Reporter Metrics + +| Metric Name | Type | Labels | Description | +|-------------|------|--------|-------------| +| `iceberg_metrics_reporter_active` | Counter | - | Iceberg metrics reporter status (value 1 means reporter is active) | +| `iceberg_metrics_report_errors_total` | Counter | type | Total number of errors while processing metrics reports | + +### HTTP/REST API Metrics + +These metrics track HTTP requests to the REST catalog API. + +| Metric Name | Type | Labels | Description | +|-------------|------|--------|-------------| +| `iceberg_http_requests_total` | Counter | method, route | Total number of HTTP requests | +| `iceberg_http_responses_total` | Counter | method, route, status_class | Total number of HTTP responses by status class | +| `iceberg_http_request_duration_seconds` | Histogram | method, route | HTTP request duration in seconds | +| `iceberg_http_requests_in_flight` | Gauge | - | Number of HTTP requests currently being processed | +| `iceberg_http_response_size_bytes_total` | Counter | method, route | HTTP response size in bytes | + +### InsertWatch (S3 Watch) Metrics + +These metrics track S3 event-driven file insertions. + +| Metric Name | Type | Labels | Description | +|-------------|------|--------|-------------| +| `ice_watch_messages_received_total` | Counter | table, queue, instance | Total SQS messages received | +| `ice_watch_files_inserted_total` | Counter | table, queue, instance | Total files inserted from S3 events | +| `ice_watch_transactions_total` | Counter | table, queue, instance | Total insert transactions committed | +| `ice_watch_transactions_failed_total` | Counter | table, queue, instance | Total failed transactions | +| `ice_watch_retry_attempts_total` | Counter | table, queue, instance | Total retry attempts | +| `ice_watch_events_by_status_total` | Counter | table, queue, instance, status | Events by match status (matched/unmatched) | + +### Maintenance Metrics + +These metrics track background maintenance operations. + +| Metric Name | Type | Labels | Description | +|-------------|------|--------|-------------| +| `ice_maintenance_runs_total` | Counter | status | Total number of maintenance runs | +| `ice_maintenance_duration_seconds` | Histogram | - | Duration of maintenance run in seconds | +| `ice_maintenance_in_progress` | Gauge | - | Whether maintenance is currently running (1 = running, 0 = idle) | +| `ice_maintenance_last_run_timestamp` | Gauge | - | Unix timestamp of the last maintenance run | +| `ice_maintenance_start_timestamp` | Gauge | - | Unix timestamp when current maintenance started | +| `ice_maintenance_skipped_total` | Counter | - | Times maintenance was skipped (already in maintenance mode) | +| `ice_maintenance_orphan_files_found_total` | Counter | table | Total orphaned files discovered | +| `ice_maintenance_orphan_files_deleted_total` | Counter | table | Total orphaned files successfully deleted | +| `ice_maintenance_orphan_files_excluded_total` | Counter | table | Files excluded by whitelist | +| `ice_maintenance_orphan_delete_failures_total` | Counter | table | Files that failed to delete | +| `ice_maintenance_compaction_files_merged_total` | Counter | table | Total input files merged during compaction | +| `ice_maintenance_compaction_output_files_total` | Counter | table | Total output files produced after merge | +| `ice_maintenance_compaction_bytes_read_total` | Counter | table | Total bytes read during compaction | +| `ice_maintenance_compaction_bytes_written_total` | Counter | table | Total bytes written during compaction | + +## Label Descriptions + +| Label | Description | +|-------|-------------| +| `catalog` | Catalog name (default: "default") | +| `namespace` | Namespace/schema name (e.g., "db.schema") | +| `table` | Table name | +| `operation` | Commit operation type (e.g., "append", "replace", "overwrite") | +| `method` | HTTP method (GET, POST, DELETE, etc.) | +| `route` | REST API route name (e.g., LIST_NAMESPACES, LOAD_TABLE) | +| `status_class` | HTTP status code class (200, 400, 404, 500, etc.) | +| `status` | Maintenance run status (success, failure) or event match status | +| `type` | Error type for metrics report errors | +| `queue` | SQS queue name for InsertWatch | +| `instance` | Instance identifier for InsertWatch | + + +## Grafana Dashboard + +Import the dashboard from `iceberg-metrics-dashboard.json` in this directory. + diff --git a/examples/grafana/screenshots/commit-delete-metrics.png b/examples/grafana/screenshots/commit-delete-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..622db1320ec720138bc8b9353bde79da9bc7648d GIT binary patch literal 239661 zcmeFZcT^MI-Zl(~AflomARtAgNR4#qO{5p;9qCd-?*Ty(r3#T=BGN&6FCn7zD$M|) zNw1+3Ae3*o&v~Bbtna&h-tYL|yVhZ?Fq4_tGkf;_)$4cdyw^~Zzj>YdIt~ubO+|&L znm9PbbvQWqc-IJkBVsqIU*q81P_~nm)lig`WzcW~+t@i;;JqQWl+h^9inv~4^0e%85kolb?Q(NHiZ-+%1u;9v zxnyjzxWyy7&Q=8+>Ix=ENGiLlGPX}^0+^;(}?+3W$~ zyTzgV=ld6Yv=@yFOOFVa%HHaTYf+G9MN%zoDjqI8X4VZmddQW(G|>8ntd_#Q;aB7sTDnkVzx{CUvVxZ=X$(KwJSwo#Ekmn0$c#)yHq|>&Ks6 zWYE7r@ZIlxgW(GgrZU*aRkmQ@d38_b2IIsWK^+M?H0*m3{g?hHG?;tu<{QO-v1LYW zzjJy0L#>Wa(fAR~+@k)ETL-UC_MnHBfi4{RtKr!BX$@NTd6h|=C$roxbhpsNwqBu} zwKHIAU47ZA3yKyJjuEiD>*r4&&F@Nv_x*^a_M$F%&E9Q%kwhO~YWMnh>DJ+mbQo6l zFtqF@js1JBZ?|47mqEv_ejzF%y^s3{bT4&&$sC0Esmnz z?8$=mg5lMOLXbK?m-_KzH&*4mo>r=4U(Q>n7Vq@M-hXw)?-}nJC&yXD$RGImlTBev zq?5R2>W>juW+6?rwU|zC?)@h?*vVfUzqY;m-`>j;R>Yc78?v_^vc?iC(6|x$zVl!#ja7g2wlGU1|BiiV^waT2 zgg0g9e-OPS5V^`FMfpj2khh9-D)eV2>!9+=%_>?*@Zk#_t?x-mY)h}odAKRwGh41E z-AIihx6ILJ){PMO=GpwV+R|i9KJ6W!3k4d-+vOTv4DmMGkz)XnX|&!i&m+52>hs%M z?T9-s@nu8ATcBSbMn1HiE$7YV8`(E}+Xib4O%F-mt9%w1q?L$JY~`H0F-JN_ zFh@5>WI?mTQy%t|aTU6uIGwFP3i~L}&RG0#laZFygxT|h#D~#N7Asq~+rr754=Cf5 zyF|M>I`z5+I$>-nca_6gnd2CIUo@$xbMeE}b8nJ8gua%xX%Jpt1pHVt5VXc}sF-9A6 zfUX@A*?4qFdgx0M5{BrwAHxz8li{4-#)eE=yeXAm(WT95+P4@91g^&gf1#8RLTdehufKC+xnGvi} zi8T_%MqkOlkT=~vA<`ojm7D$?|JinRdX-}sX6s}7rF*4!zmKzTgVeokaZbR0+RiqNPz*0%2fu+#8U5-qY^x};nIRo-IjhN=CwVHi0+`6=%e*n^}&v0;y;reqRZ zR~x6~s9v?yDY1@#{t#hW_t(q1eOdiI85oC5tzX^WYWK*Ng}5IZKK5>zhlKZ<>YMEpVQVf+_irHqN@!ie`b8>zBRO;Q`|F|;AE_3 z%xw_p9=a7?7;nfytSezE;H5KV|8s3_6pU0cbk7{g@fbE-Gi;kUtk|x2>q)VmP>R*e zFbK^0hOij$`+g9CM9eTux_D;Iv`?r$OH~qt0Qwc+ue_bmpj6a7RNPv8Yy@*w9`iy| z9)d!Z)?71hNaTLsYz&E#VWG8 zstV@v(Sz0<`5hGZ*Tqd;nwH<}KSg#$LSs@{1!lwr&?5)KgLy^+Z_1Xnf++|%4U z2K-^*hFRZV+j56(C(6!=Ylg-~x6am1)-!c$e>U`>6KP6(jO`q0hP^flw??5zypnWy ze`Db4v(~7Z)Ey!4j8gq*ppg&uI6Y0YG_dDlb8o61a}5)|y4VhR5e!9 zL+)*tEk57KYK*xWVc@lV<9$TK$(}J==65ixz1qaI~ ziGm=K^F@rFGNuk4tyiH3FHLM3462+j?m;bOJKSNUlHypg z1L9TXaXnON)XcZ$srA0i&G=pG(`GZ=8;@cHS8Jz6X(QEcCWSYGJ^Ag>>}J+U(rTY6^9+zzlMX4OO10C z*uw=r61X(~z5fjN5zdvrj^p9rgxTTX|MMC(V0-!Z0r*_D`N#H3+*=$%;2SCM@%e=J zUsn^?eY)~r`}n|paB!rxWEB;Gt(K*mwY9VROR$Gr6?GzTfXGF`z#RvNg8A};tEkDe z4cvdsPFvqYUsXlK670lnZUwfm=Js)Nxx5dKn2!js>tyX=&fw$Z=EwGO#x4r7z&WM8}g`@aXTH6PAW6Cp)+?w+1&Xu8jk4M+SLqgtIc!$Id z4Gl3r;9_8Bm{RZ-Qn*HX5>A`{=H07zuNbd=e)Vyr**)WcYS|QR;zj6f4niCU)bZ{0 z@*O}Dg8I{hp-i&vj?cfn!NI%6@ao_H_m!DKK_&#tOoyu$^Y6F5Z1DnL!DX2Q7Z?Aw z6wbf>Z?@t#L(5skJpyjLf4l8pPkV*$eNONvPX=0Z6XTLjf+Eak8UOGGz>9|I!(c*+vA@fO{Y2D8 zaxs}rqnMl~s`}4ZW1+0-oh_1qiOdm{-9`~h9ivPZC$_ywiZ%m=Op0+#vHAIr3XF5A z?n?w5N!BCI&%(N*>cPAnB3>IwBLzkRx*c3vxoIOMIy0N7%4LU&!0$rnlN{4`>Xph6 za2ox;uWrFTCJfJ6>kvaZsbYs?T0GPaGTzJmPF5%qBg+lS_cU8g^W3XQaK67ox=3w& zY4gc96Y#l(Ls#s>JcF$hdwF5^rO3uj8DjTk?${fvd<|1?Z_zM-5%5kElG?}qhm2hx zc`6%ykG7>Ne7a=3&nzgomjX9@K~^?c9{E4evfv9-hC!A(5*iW1-DS`{hR-_P)qehk z+6AhkZjC!51zMU#Ue?F+F|TV@!;>*+0gdYvoQyM&+UKU|5tiBJL{0;)0$PsrZV5dV zwX`R9%EWWyXYnx*>>>4h(qa9<&r{yhK380m^B#OmL_PhMT$|O>W(V(|SmD!Jy!X5< zaLpY1p}N6{lGflWh^~)p^NVK!Tipd(dCYc0McLA4&om1(J%t9v1&PMcxTPOLBNVTZ zsJZA>_#G7W3`lO>DncSI9@l3n48Gb-v<&h8dbJ|tPYgN{%ZEo~lZhaIvp80D=MT|z zXUc`h{M!{LM_Dlv4@}Hfe_8x2G*LXh zM!&`xHHU_Z`+KqJetC1Z30|hi9%S2w+MSP)tV3;vz;%oJOF+0Q+PNyiU9s~71^T~y zQGLSajKas5ncZP^QLDvnYiIwuPz@q9Ea?2b2c+7v(`3Y9HBP6Wm{9RbO_;KadJ2ER zM|K@uXOMEb#F8*YgHa6n2Lifki9L>&_)>?*4cf14z7lwtS3U zZ@FypRoH%pKN>1e7ie4|JYNRhD&1X7H0w-skW3uTQ@`r%iLA6msG$)UX*TVnTZ&MY z>W%SA`Dq9^$8o%3eDgcr9|)9Qh16$dm!Q*~ylmxVXPe&7pZmLrJqtyMRuOM$zjr}x zM7t)1`@B@==y*M(q-NGe?UmC-ZcZs}O-C6Xz@@(%?qX(G;e8eMi}JY_@1d+#=*i?= znavYiRMzLu*>XM4+=j*_>=&4XsAjx|tk&K}*>}+g-LFy|YKGXdnSRMOfQdMTXcy{_ zDu$@7J0nB4no*BH)y^!X@Sj?Afh*;&oEmt+(<)S=&Q?;5aNTP5^WH2Q)W&!OlU92# z?%75+EQszS`+%ga-uHf4<~o?T|1+gTUqc|+ai0!6SQm&b)N#l_(y_Okd9m6df)Ed) zjl%a%9IDUt%+{GuwP5$!w!^@tcH)^7R%<26<&63L)nUzw5wfME%-)SI4;2=*B<^Hh z>vC1J@fJdF%X#-#*jA&HSEgl{UK-`SjOrrXJ^H{g_C6ziH&eeSG(Y;U|H&kzTu#Ck z%De0?e2x;Gjetw3{e%7b-pw^cLPK374CjaG`^4PqpH-m2qHOdJx zE>kI~XGu5sNl9ALi1w?zpEg^f)6c4wM0I%X$*TG(sM+2?%hLpD=`U$&sd|Lbg~q6c zH0sX7M6A{HMd1|b&!X=$f`xTXHb=7j&FW_g=D@5F+&Msk<9Z8Lz^rG6QrJVTO?LM` zhz&~=f}bYNN|=AU(d8-Y5rRn(*isrWEyYsT8G-vi;9E57EP3FGDrpv#`wttv&X}xw zv_GT>tTzyK&U`WApcSq08nA@6S4)%S$VcB}hXtOOi<#xT1&qb1cQQqg1hc*U5OH%DZZH{k6DDe#sF1LQBhV`jAZ)>3~4ywa}(9R5pO?_ z?r!o296hNQFbk{{@!fsGX1oynfe4M6bcgM)QKerC+K!pY@jD2QZZYt`rwHSQ_oh^L zLUv}8ZRtIus)j?+9S#8}`)q^jKdts>+0v4okmut5a2AK3P0Q#M5l?$o;h_4|&gpDB zljEHZdZvy8Ve8$6@myGR@HuU|sCWNVDG)Ej@yXvBKiVWX-u{l6LXg*9`P0ctnJp=y z5&Q!6+BCwyO|o*4dzZ~E&yGX)^w%q=77k)cPN+kPqHW4+Xyj{bdzFOUkP2)0lf>1b zdSxSn#>0rc27hQB4gZ%_ZSNVo6Cbkl*Bmfp{&~nq0a^NsW(Zc$ZsLpSS66t4<5X^P zfm#VIpkro`gL;p(N$wYj9JT5L?F((AQmxtsz;>Xf&KIC<*~2}h$p>a>hYkH+qT>!x zTlkpkiyX>}-Gj>}h+j0zfrcr{Kf>4dmbv(Jx)0=V57q{=%sTNg6n!X#@e1sLi{H_!y0#u)`;rz0L@D*$pbOArE1fS)l^ctk0Y1{z}^c zbW=d=;1`DjU&!+ZK;UJy7)~-NacR6+BIzH(0!}hf?SJBvuRSGZ(hu>E{pKyA(=o$u zd%!~J4@H5r;U?Z0BGx6w-ctulPk>-p0SJm;B+Q>w28?5!tDTA~O32}_^IBd%Q)mg1 zQ`yT_w6tFOQeY#jjaM~Qs}yfPRMrw*yr|jVa*O-Kcp=FZZHrq7b)}+& z&1{cVx1|Zdt`M`jB8vAnMW0xOGz$4n!Rzl)Jd2`c2VCp&;WqVEgwLn+CeWQXyfuW^ z1)8I%1wBcR9Fn-rg3fZ9CJUlA16%4bfljspA}%iw0mqBWyp~IMkIyhyyvY8@CFlp^ zE4%3Znq~4HK2%>k9Am0=xGiInI$h?T(zr5KXx<`E|TQR+GVvxK_R!6Y5 z!!Ji!$akYWuXoLVt2j;q(wMraLG;55#v#yRLRJL_;lDiaEdGWr7MrQ-BCfFPh%qOd zT<$Yj)0P@|!3KH)lMG6Tq!RtMNeLrDPc-(xlVbsFsps`{ie+2|2y zLg%}{vwZbc_OfS8czE1Y*Q{p4Od(&`&ZWHe*%LzS1qs$3r%Q^wx^&_4=F%Gctu-dKsSh7XH z;3az%-z_Dd_hf7_0m3=vxCfAD^UsU@e6@m`UbK8ui^D1md+o2>$!j>5Vl8h>HIV$L zQWzZS^3zKMeBd;#_)+C=W~yIrU;nZVla$M4B-qmo@t@+J<~b|nsPn%-H+%P^Dy$!3 zn#xTSVj1i2a_DNJ-PzGIUPggkA09v(i&*M~9JQ6bHYZkV%Krqaif8#SON74e2%Qvi z)Mp*eRsDFA2H7?tZr!b&%kC4$q}VIL!%Sn_m!g`ho(hyV&_6QczZhLqkAoN95yc5@ z72AxbS~-E97BzuPt28xpl$=u7^|-mhZqf7h#3s$28`s+|9PL;h9)g~b#Lt$9`WBVZ zi=QMKLk)}*6iJ-4v^vasm1%|$GkRdO zH^n0RHZlPSGm`uGY*k|wq3xk-N;H2WIB&i(E4m=`$C0M%Y$THVN&ZPB5M%Fd?n1r6+Z%MufHB~4VpYrdP+=WWk4hDAU|#Ie5nOMW+0%hR5$7j8Q`6<&Id=Dl z2T~5S^3)Uo1JZz!VU|Kg=@e>1-`+IkFZySoRg>Bqa8h*~$7SSIIoY7fH(q6zWzej# zTP@_FxY8x84ave-6eB#%KYfRk7>s;ZDGocjU znCp;*I(qiVLi#f7>Tq6;M{?f6WW9T?xN(SoFy2+`eCT#DRCf_P@g-wpvi|#*^%#~A zAY@Vja@7Yxr>+_zp3V;?FPczOftlF)59Q(-Pb$f`zQh)lgsVM6Uf;Vy@HFVA6h;|v z#|8Eeaz{a*PIWqWb%r9X=+#lF8iCW}odA!V4&<`Ba71 zx|egba5TYVbtu_MK87xKm;7^4c`Cv3vJA$(oCtk$O-P_??JxkysQ8fHo7Kop_W_-R zyKI_Cs?&)#nn3g`?V#o(?Vn$cVc9D2Lj^1sU+zBA=%iC)*XF#l9T(adeDzP_{@cM% zlV?jTp%Rd1RN{W06$7H}zFcOo@9xN2?ZkyHd-lXWOpK%vZ7aBzaoXfm^DCVk-1A8q zvQd~UuN30HI<%Or)8mDU@wyDa>_^>joicR+xpJ71fD?8BhH@RPiw$M3@ZKs#`>7D2 zQy3&YyNv<7B?2YsBWkY4GG5Z*Yoe zwA;^DbldB@Xny?5d7Crf=ofZqKB^>xg5u794A90M$H*V=iL$EF7^x zkTTN?A#F)Eiby9rHej62)Q&NPgWTr#5-OC%Vq^t)~Y8RPK z7i0u|-7M{zOc^O2M#IwA3of<}UK*8VoGoVf)BTX6k_h#JDqKQvTjz6fH9J}MMDKZ3fA>U3>z_fz8ccawWaoo_og&$=!38vKRAS{HuYUC%IE2-}qGtarBs)tO~`bh<*+x{}zkHYSgurw2&0 z#5SiKwRFSXCHQE;Pnh)K)`e<~u+HE>x+rClL&O$#Fp+~lR#HAXt)k=Df4t!UOKhq) zl&#mLDzq5uXi(WhwJaCnx!x>DJ_D+INc#s>h&ma3MwXVd>r^0TvVY%H0WIF?12vF` zUN7r;{y{U0cv z6i)Cva)uUhi?9S5*+1$%X(A30`n^L_o#CGip2bxKmzc54#r8j#d7$^L%u*r#bJU;` zia#2>|4jXVrv5)#|L?2b|Ls~I{DiqKPZ?%9l+*aLMS--OVe;PZ^Nfq@PrH77mtWY< zl)VsaGk;&_cfV{U{C9<=+qig?lzMuv^nrTjZ3)$s5x0#o9%0WlZTwl+{{@`6u=u`YAY;EGx^>jD#+u`Iy^>lWGFCC3vTqe>5!XW-yLA?5$htM=SVsS@TQ1Dk zUOtw9YvJ2qN~T;JOy<^H=RWiO_V#bC9YqzPP%*M0{?6lw3!z5({{^imxflG!pi-C! z-8kogsCOgNiy^z0{#$Rv`0T5?@zk^!5n4Ay_;($^!86A{#h+7jA$hI%`(CyZAH6R! zx=T7)jQ_uM@7uK_qE_fE9RbwlcRnik9pyHeUhN~MIwPW`--Eqx?rf92V{H+?!P58J zi|&ACSxcOsE4b8>_WpJZZeM#i%X(Wq#Q&a3%5N`XG`$?C&mU%;nUfy=7A&|83@1Yd zKIU-}|8FmP+Vl!m*C6+xWlHJs@9W46k*{Dq?6xY3`PDz-2{!^ z0n!<>l7U#QK&(@#<8;%irU}9F-^aee%1oEq01{{+n~b33c%Uv}V9aaV&8QgnZet2l zQeKXQ`lywxZ9#Dvre;**m_WtHnz~qb4wO2b+02^5$BV%Tx}|X$$LPUa zPx(vq&(XzrNmOEoXZ2o{1FFln9jZ;yEPxb)Uh~YL8m4%6cy^9e9s0GH$TV3QHJqo` zWs^6MXcn08=%Pe+3Y%jeu{ma3xIW+XcQ2=xaH-(rN@e&MF<<9c2{?fwkU?+MUs(#e zfM{NlOz0s^ET6o+)k@QwN^l>XsV|dNC>IK?JEo< zAtzPZ`P$#6ep?-IVY@WyX8xg!2~(Ek#YEFbtI?QaiVI_ zDzXid{F~{-;s3Goz@Nqjznn(KbP_0Pbn)`}t^d z(!AZiQ95PU!d4hGI^9ro^m#L}cfRt^l6b>sH(~;}%Y3@^1uf%ygLf$W-RIk5J~As)PK@dgkokxB zkMojeo4;Pc$Jz^Yr(WNmwxv0VTv-{+YJaGj!jIb)>tO`V^<2cfCbZa;>v#!d;m68< z2Oia>;R6PK^Ps~YC=Z*i+ditcs8vIp)u=aMR^|e2c&)oX_UCbm_MBnFt!f+=ce)1> zd3yA|#lQ#&ry0ru&)tU>%DHs_>{RYb2Z{JmzMbg0_FLbsf+2uRbE|e>vCJhCAJg-T zx}M-S@xj;Y{~Vz{Bf$?Qjb14|3K?SclkNj&nnlk{x{~=4?XY`4=!eRla~)&15Suye zzGy&@m!VHY$MC(Q^c>=yC5IQ&W~XaZS$$8-^YEM1Ml9Yj zakp6zZS0x=N0L$W>iDzpyKD=6nU;%w%fzK3@amTtDrx-foO@*cz(`qw<<$SCx4}n$ z1u}Z%TSRDe@q@^wkqWplOgE*#2vMk2e^w`s5T)mT@n)gVzsO)~S4kA8ocejEa=)D2 zd*1;Q#2p0C_pTcHn(hwz(pc{qqg8G87&oj<_eTSQeHt6~WDZl-$(?!$9T1W z9;f~jd5N?AhxI${Vw#oDXuEz)ITNfYtxrc+80a^Uiw zfxrNcpF)+Ue??JTS6#Ulv-{E)XL_S@fIhXz>;6-_R@~=7E<@So`q`V!Qp7o9RO-+2 z)ThOKHoZ*iKZBTM!*05uJf%7EW}au~-#=b2t8yG2bKFabrsGq&PTuf+at|OEa9klg zh>ejv{<=Sm<59^{oVF>)WYVzjcDNzD%=a15)pT6B@S5bp_GCr9REQv{TrHlx8s)V; zyVWn70$ZqdPFF!~o6o0(?<3zdwMSC%DiJLYf5C{5xVbHK+DYh~^fi<&q?0QH+^0aV zm;&Nyo)M*I2WtWejp5S|{nSqsktC9SW=s94RAnT;)zvZ}L^};ipLH%(obR#hP$7@< zU#~h%XXdrvf803UTWQ_H&6QGnhfDBtxvuJ~i`6`O)s~O+HDEr-9BE#3-wqx(L#vW8 z+2rf_HNQ?`=DOuB_lS=^HC2M88u=9@qYD!t0jfNz=0BueC(|@?WSoBWU zOg&ws`gnIkXeNLes1IH{=KPMPw3h()YYx`cwx0-`r-@=zb5mJ-96o#JD_V~Fo%^RP z6H?dC1ie7?Ci9t8C8;X1-YU`h-d^3i(}b1W-JaKM(kJRl1R>Hyy?+LDRf8|+cx``@ zYHbE#V^G=ETBq%z<6|cfj5q^;GRj8Nin2_AJAa%%C{@3Z=uRkRh!NYZo1AUZ$TV9t z*=zCN9bbkiAS7h_GHpeXB9=>5TXwy>r}bdb0Zq*=O0xiV@V?(@mC4HZTU|HgI?}E~ z9hEbxtlC>~R)5hpRh!^@4CcojGA<1dlB&CaJ{N}~uzU3Yq0jaShal9x&*UfV) zZi!utZHj?4!^k-jlh^iZW|E#m`ylIk`AE~CaluA+Y(TpNM+FV3o8yLX^4|vJw`VQ` z-@}wS@fJwur{@5M?%Q|V;Nb+K5h|%%zLn%4Rbhf!Sy+;bSnio)z6h+)dcEBi5r1O^ ztk*oVo9?kP)?Gq?Ut4=tCbzo?@0+54-)6o-aG0R0HUwv>@no?n@N^L*&vSV?gCqgw>O?^q5Td{Ph{a2mh`!Pu+3UWvi|q(%f{?cPW@v3$(A_4Stf!fs%$3x^InvdmJ|bbkrEf>lf0qL?1pnoVbK3kvup7w{08hIpeNRqt z+^tPkSD~De%>Fzeo?0#Epuq;p5*AeHT%7jARR--|SeUIwo6b}mKZ!jI`o6zIP+Kis z^!c(dUBI3J)ZvFa&R<&ZiTelMDXuezP9eYe>Jf;HB0YBrq&-cFnl9!7)T;6e<}#F0 zd$Zq<0@1dYMXF+2`1aMbniU3H44sM9rrsd#uUV-8ASK8#9G%$R*J_($NBhwuo*?%Y z6Jg=SpY{gW@NCM7@YZjMH(>Zi$2{weFx<{A69n&Mb)@PwLK^9m&G5&X&j^JB7C!Wv zJw`EoRKG7dTuf2rfKkt7j|gPMH2c%@k6L&jfB@qTgzb~nd)~O2WB!6O?R9|s`BA56 zhKF2T!tI8~befh{g%}L~HOr~!5h4Wsm zX8sHF%j94+OLElyBT0V6ob!v{EhKTdjGpKDT1*J7e@51-#8tJ=^oZ&8Mrt*9tXg$v z*I%?+XQhFno$qp;XS%vX<*a49x~n!vsiui=ckmI${9=_Lo7`J$>^o~HVsR|>&LG~H ztd>Q4T8tP%j9I~&<+?a#kHN{q)U9cxTYd)7uFND$*#2Ph6!j&WtakGcJ1jk|Wb3Za)O}j86DnZu<#-FEfCMAw1uD;b}kIkhSMIxq@JW z2xhfK0F2l9Q(AHX@@`euWmfGF|0$6u6&=5`#( z3#yn$;|-n>r*-p$du^tvo##NUI>$U~qFv2Q><8s-kLxvds9Y%4I2*R7q6`aT+8;|I zo+tb}JR%sM^b$x)qyXYC!RH1=`Y%acjq^hH5>vw&Q=U$jxjGGYuZZNRBx~BJ7|jxB za_Dle;`6It)dmX(p4Mr_!sG0akfnapIhM9{TYkb@YsGzTdC;=yvxAlFLxA(95TQF5 ztO>q7V#H=z^v*%1Hr;Q3V~r^%$VS(w+HskwSkfT@u452FHo4jCY5J0ZoI^Wmy4I6y zyxg*J>aJH{CPUd&xnK1J%X?E~_0VZt&Y;fDX3~+O3sTCfa_In1BCHTZc$qs)dN~A= zRn@|1M)w4rhK9}}3vZ6uHwIF8ztI{sWd(bB56_;yX=qOoSnJ%=u}|SXQ5;8u#~7NE zUX>T??=4OYWOvY*s7h|#JKxZbw>z{D_rJcns6AXxK1FzQm17HmnD~8~I7<$|Y%u`a zP5R0+h3Tf+l%9ZQLxaI2AM~V<61+k6X_4GOs$@?Tu>R$lZPXj%WXj{ZdZIENIFP=G0XEvP30COs!G*a0A(6BDdvQk`-%Y9@ssyllwK= zxpv?^5H7JF(2Teg<@iItUz}sJwF@;38wu|=*8?tLn*9)bi~eQ!K@#+dq%2{KSNPVX zg)YEBW!*Xh#~(K)OJbbzxZZC7_zv{Qw8zWaOrhNDWb`JHxFEW%>~En+DZFa{7K`c$ z$KYd_BXpCyDX=7`Hh3|It>wu8=|(zNHQBbI8DUd>gL+cvMeTC8?gxT{)a05BxEXXn z@`8!s(MIt`NxJFkqzSrlOM3XCZRxk0f&$#wPA?&3 z4s!YC)3x^XlAZ^a^6ETiO^W-1EB7ZQa|sF2T_F@r(l5ZI5Dg)=9Y-pFz6{79L9x?| z3H3`7pV(1A%lZk`klz7y4;&-*LXb=Ui4luB6*%FHsSdNr*Q$MQ=#^h+IP+8(=&;G% zhKmO_G8#tjxh99;pQZhN=OJNFc)VdOtKV*5i5HN8>@ICf|DMmsAO#?_63t|jjo#}I zy{BDk>(!#(Pqc2oBKFpe6YlEtO}C%gobaMB@|>V~7~_z3BGW-EfiMB+(x{r*dZ9aF z+Qw6<_t2^?wO5^M9H%%pMXPQ+*(g5V?Rj+$PO16zkf5tur$>&gI!`6TVWP_K_+tT? z)v>7Zr2RZDVcpkoA%kgb{Fuw8U!A>JH7z0Z@6i%LuwtMaZFiooS9!qel+Qlh=oD6B7UbGi5TmV0iu;fr&)2W_s%Lfwl zSImI0h$M??hct>$hsT3^-o?sJYWc#P>Y^S{ROzesBt-S{;cU5o)*wHi5whpbQI6-6 zFAx3D1X$q1@%|=ZW!3ltjcKH5YX4n}At=Dv!43sUsk_roXCD=gX7CXmPunt1FLMH- zw}f~6e~L*OZ`Nc8yBl`vahea%6!~_MTBlu~9VD8NeS46INmGC02Qd2SHnr0R?GDzw zt&+6F9~!$DVXMGMIXcJ!k;#pGwXAhPVG(0MNXAOfSk=?GfJG7sv)%i4L)knXLlTK1 zEpVsx#KoNs*iYqH+}D1|_IwQiJVa<9`t{=!wF{-b$(RVFQ^#)#3C3RMMF4XKLy5aB zl#S%;@EmPBSKn2A1u`nB8%N)?s=Z$Lp%lhFYPoioT>&h({rd0FSK!!_OVn~FeHL9j zX0lYjqF;7P7YK9#UNz-YvT%V|QG<$w<)}=dB?tgw)X{A4$1+Q+taLR&!f!=&BS}xZO8;1d`&;rs7 zlVr)FnxZ}N#MLid4JN@%&K}R-bCRVqxAjPKN@%1)w+49~lcOZTs!Cn^*kpqkcluoY$pGyJI>p2N=~{ z?}(~tn-UNtLeDr)FqoLyE$dcdD0W>*?%GIWP)D05udbt~5$;o5+yh=s(=BH4)rn;; z=b~z(#|yZl6L5)|qaVs#s2(kKaZ4|U;hN;rGDxa{T4#AVviEPx3c+vw^3cdj53R*I z5PO4)H+gT=FSCN|{`qi*=!+6usbU5g_M>HJ?I3KfO7GL?S`57;%j!^$kSf5V zaxU7Se^O?)Ch)GL0z8njXWDOTLwvTEY=HGO0aiXBrLfT1lItpepXl7C0ct8oM`|v$ z<)+|ZDz08$`_`w5=^|0fHJ=w2x}g0?U1-wUOhXF|D2lCRYG6l<^_G4W5n9w6{Q*EP zuR0P}TUbs`gRBvjh;ag(*H z3YkcpFI-tgm4*Rq^5Sx#mMIo=)AqEvG*YAdXrK6x>B#+yDu-n9z4G^d|S*ua(K!Ll8W+`=-*BCR|j2 z-V3O%SPE2h@eWJ>4F$38+Ft^t;!>(v-vB)|8kihh+7eRiNHpW^nN}J^)j&XGEK#w2 zsY1z4Q8WTVW^*I)V=Q%8;S6Decl4je~C#%y{fp7B5_r(aP8Oh325BeV#B zN@Ax!EHyV)ZJ7bkJ9jUkI5h_nmcWC?f83V)U%v&A--S$hLz`-Mc2b(N?m(-G%vCj1 z>LML<-Iy(nu_pxU_yB%s^H9Qj^IS2d_GP{*dO>v`@QN|6&2RkSdvcSO9nngF_nkcd zMX|e#nDT9YJmP5DWEC6$Gcu`%Q=76Qex0 z0NcEfrt}8*Por=Oc!xn$+)pRdkHE@=6xFRS`s@$oIDvo5sZfx*te*P& z?qtkn090WrNv{<5=9M+(tY;+gi(8!4%R=rq8`koqvLvC@%sNH!P)oH%aC!o#p>}V8emH^Q1csqt+4W87Bp}T|g6x=OSSVjHD ztYWdMc~;hI=b9e3C$S9HKRz_?o@0xgyu19BC8+pa-c)9^9xtq7hH4=D%UW>c668ge zsAkL5qUCQAS8fKN*iCPCiAWn5o7#{5f^kI0ahFylMyZ7qtJSReYL5uB8I>GKLO$Dq zZ_#oHO(%7H@tMz=zK(8jwU_K5eo|oD#&V@YJ_LJ7;&n)H%$<#BhXa7*Vupd`(rD4o zxIQ%jui|ItNHckrxqS#mg29j+e(@@UbTPo>rm&dcH$4t#?yYZfJqI@%aFlwkFQ5~IwJ4~z?(2~qUWP#^d?(tOcGdOnX}JI7 znTXelecXO^%cVUpbnxFxxM_tfNpE<`Qfx6(C?|SsJut94ZcE=omp4~evsy_NRlB}- zrC0@hkj!frZPk14Y|#|Jd;o0Us-?-=29aEu+x=x+D@%QMpcIn~5&8HyBk? z-&lIqNaT#Xb)qSgDI=$1eF`w1HGtLj7?_R=avrs#vvqRQA!S)(q6Atybhb|gwd$Xw z8l*`x16wpp%yfT{$QswpGu09H+4inAlZlG*m!bGS$z=(v{a{nnU6AfB%>7yN?5Zs+ zXbxY+yGPu628{PXPy;)@Os`xzU3xJrRHoFGt4d=Psg)^P6!|jr_E~R^8ug3OSJi6o zxmrw$TBMuF6fUB9(z)wgOFexg@k+@~5Iqa1>LNFvEiJAtr>&eKuizg;?t zs)j=v`omx?KQ3vq&-c+))&#z5oF?6M!ca}3lZnlRhxy0PMM&cU%u~kg}o>1V& z6+K)ud>Pm7+=6D19V8NUPgq~~%LqdQ*%A1MN_RrRiv&1{V^uJ|_xEoke@{

    q|! zbA*Bm5|lwF=A$u`tI8pFeTL1@&KU8{F7KZK`DS6$SjqshJq`8s6=w(xtA0evR(J}ZKKTjIvW4HJ|D$H38vbC*x&8vjQEuy`3 zI7b;=~#)J26;1qDAt?Qqx?OYfk07= zGsUH3_;MZbgl7iHXkP%X4&qHAlxsSU*IFr=&*6P)YqP~7ir2RPM*V88*iy36+Z%&K z3k@+sEhaG56`$4Y@`LADVO_aff+;e5z-ri3p%yr>P-ducnZe1+0+W}mj-adX+FXtU zmg@O#j+$yR1W8vqbq$5`I-e?3iUn?X4B|Z#v3pJh06CzW{B9QL(Cbs?69>RDk@FF) zfp-*zg9XlT`(13y6`S#MAD9z9;812(CWV!me0T5WHUhr^NG0Kzw$KlO+97S}=G&wA zI|8>`)%D%fy|a^#_3!)1Cm>{67k@K z+(b1HS-5wCnM=1BFkU!Cz$cupA7DOeSk9Z`|2-p%>C1xl0AxQv zl*FyoBQ)jM1RY@m;^Ww*fIBB$>w(fV4Rzu8hjRyhSt)omS+hUM1 zA^;>TVk17N=}oFSv56SGj=5*ebS~5D9sj2r1pjTTzBg|}d6c_S_j}#y?9a$tGPu$T zY7_8MEVk(^_Um=u@u0aK(K1-3H)!m!dOxHLFm8SKKfJqOzqKMhP`<-=E$a46wBwwj zZTDQOw1zTMhsC;0_e;1mP}57w_^V#(hA;{lIVRV)+NPf_GL4_Qj>`i!7+<#0Q(3~j zQZT$s|3QdtcexsuhRZYNPN5Rlc!md6ETsHZ*44-9UO-4h(5uQa$3~&`z$j4TNin6U#KCtpjG%|JXZJi6X&|k5sDD3UvXQ}sEU!2x?ac^Pw43+K!01wpIz#ui=)vT8!8E8)@s+}SIClt~ zwsX!@S7brw)r&Nf4ER%(+HILOa{*D!GK+$cso^sh+hmJPmvMtGgOjRipFdlzn~FE8 za<*}u0x~YPDUrTY54cK&&DUNRK?l?Q=5k_hU()X7+_5S^U#QX}w8n1S_}mY_c}e7W z+Ng~2G}!Epti)+)aFsMr!Y!D&6J!0NL@a5@GusKR1j*caJ9V!jP7ZaOF6A~*U^<}W zW*!6GkpptY!3j^MlO=0~pPVf$QQ-N$3RC5Q-{_A5v%$8l$4P{2Ai|dNU}vFPJxT-~ zZRuG4j?w(n;b9=C(4Z1}w)p*;hJJ=qiWi)AVCAGhZPFGCe}@xM$#zF99n>zgM;fJ$ z>^^I;QLALtK6$8VSnG^oEZ6wZtqP`Ox=JC_mT9Z-F*KqHbJ}BkiC>250Sz=t)Hv>q zQhb$gcdDK;HJc2}K&@Up5bC-WB^Z_iI%nKOC+0U*tQ;Q$l*!+?gkJ@`*6S5R+HdWt za=dIs9LoyCiV0$!n{(ZEBA{WLO6E`LikuFhwBn(B56Ya1EuvU{dbcI*P6G1TcnmTl z)meaGwK2V!-+vg4o|#p(Z=G0`1)_wXaZ(YViFJG0oz??x37XCy$w{UAywcZYvv#An zyss+P)vL5s^a@a|qrZ`QBLcoW%cbv0jN+4M4Qug&yqV)vio1B-MrIA;RcxLZ8An;O z(iE+zKzvwriIbV3TQ|SN7}%1?t;xy zcm#{2Cr@Jf-u~R;omiIv7s%4Ijq+th z89Y3%ln)qp}AV%~T#VbLJrx=QqGe#1b zZNhN{?wnpbTP~@H#9Qm=AW+z#EbiHgjP}i&+)F$2g)qv4K9Ubt`86}2ZjPT|{sdG@ zhsZET3c9BEE>A2xI?nf5vZLJ2s)D9yGW?6#kn zjk=@9m&_|uH}RV9>Gx?^(hdyR4iwA3z*sA_d^cLLn`3xXLvo!`?cl zkprdn8VwC0oNw_ON$7Lil3M?GOT5Di%uJV=QIIJ! ziKNKI_1q@umG#&B8h-1)Tv{&Sls48QHRMPJ zT00AG1xPcbgkD%d_D+5rk$s~+$7Ma-(siCD(=YXS;(#W(AOA-aCA zg$$$5Ke+&5Xo#%y<8E^_(WPy^Q@ehJv0YQ;Ty#)?*=xpRP5twROcu%4+JglYg|E`h zP#P;3Duz9f5KQzldB>MRsi=BrzEwu$Hq^39+q1ty3F`W`3{b$BE8S1nci`~t;i0Pv z@HvSvY>gi+`R1=l#Z{mX1R`JC_*UU@srAXliTw>>+yx(^B=r(oBU2Skm}H%cZ+Yrx zJH85Fmi;6eFVmlh0NQ;E zf1;tazq_Ba!+wG%3QHiX@a?;%bLRLx7TUIZwYEtChDX4D4`|0CpOSMwkvC>Bth{wC z<|O)>u%>`6*Vsui$eF$xFvshvXCrqR(I;(>w{g@79HUY{643~*dl>RmR3OOjK_=hg z+x0^=a(5p~?r88WvfYzH#t8&cbjZ8ktXn=#bLr3jjg7Ml}D)$2g{Yyy;_-s<_%<3D;MZiO{=ZnqA_JZw-? z+vw+x-GB4>&8JV-H7@am*p4J?U>}p;xm5W0)XPgppVpr^e*9Fx{f?tYLS8;qAisP_ z=+>!#AXwnNqpEkvFLNj#^yAojHuhd(M)77cAghL$ ztM&{nCE+|j(iX%$945Nt3u7Aa&|#NF?1}esb+jC+PExdSRT~pV1U19Kd7sk?k{A1! zPs0gz=B-*bbYr<4;P++d52?00I*+kSoBreQCI=V_-5(K{Bl`bGA}EBzz!o&eJa8*5G;9xVs-aT*Is zXi`OUcIIsOHJC7B8NyjK&(R>bYKzoH$Mo^|_3>5v#s*Wh3gzu6)3_H0C>iQGWy-Bq ziv0PbcxX09P4nJ8krm&hYKts>B&Sl$MDLG&@<3s*GU0+Am4B__RpCTyOj~IBliJa& z(@ZL23lDB1tn{6^ELx~U98%T74dQ~yX!=iFWK4YKt~sKAW5lx>IgIl}FoZ8D>bGFctA8RF|MOu1#Rp!jCJ zx6$^d*G)M&UxkvnxM{KMiiP(M_Opu#$q1{0Iol}P>0ZA?qC;!Zo&2G7cKaDCvHBI2 zjI>k(e&NdEqPv#lnNTI>`^D5}!x1R3_ZAN_xE*dBwnf z4}v4_?Xx<=!+s5WrJ_5+xtI{&xfuG)dV*$AxDc*<{Q=vd1!A|{ohOMS9nVu`!k@Lv zJQftDLOn5}J5QH!;2vYtGZpsNheQ3^_hkks)FvHMKG0ZWwpa5;;%LXycl(S-bNBEh zb{*P%a@D|-8+NRAbVW-(sUO;ygtZCD^ZaL&64~2leU# zWXe5bBXx@k?WP^IEYfE4%pEMK=3Au+sVrXar}{E=VO1_hOU*K8HFRB*2H?XV*4F~e z2pamhGMsg1yc|g2hD-%|2;Vm#E@<{C3Lg=npWHe(1I$zXTRze>i2EC#Eu3Jp+)0^} zBLbGP9%>7T%me%tvm5wFG^zSqUCZeE%O=J~oQI3ww5t62xH;3GpsEGKg4UP%t!4kvuEqq-JB^Gma zReoT`v(4Na-mR^t<;&v!;B9_uedhfcb!Lo4y6%Mt_nA`j*ukZ2sEf!lKlD?SlfS^# z{auU_sjWK+#DMiXfg`wP=N$QM6Aq`ma9_>w|&RE;6WSH z@`(jg0|Bw8at@7xly!z2^bX!IVH8i!Jt>pFefWb;)$UYsu_-<{PqtTPXMImY&j}0$_y-WHU;r^G#Bymh+=ipX)O;PI+Vx$ z+T8t*Ot#aZI;ZN%3HPgq&H)455UbNBIbIVI6l@x|rII(aRqptb#r>m(@yHD2`19bD z@?ekmNu~|X{nq?|T3cy=~Me_3y1akBvw3PF*Bx zw==(Vx9Yf^_R+{kpTQ!hlJ5C|c@KW5bVM>eSIxYJq~Bk+%(XjCt}TS`lrdN4w8paLS&rsR#Am)==lgfG=Lz<0edls{|DMV8}mo~DE z<*`Yq_|RLEaToMvCbxv#*MjWvex~*8h?a1_Nu+YT+xED;kw0(#3kS}4-W*d|O}2Q? zjj~S}`o6QkYPFJeiWZ)4FK8ar$P8ILnHqh?XC&)r`aRAu0ZkE~!_|hTb*mtChMDGv z>7Jmk6h4j(K;P@!uc5iSmPmsGPjmzVe)8bGYxv`+;HR_sR7hvXw zH}^+?11}jMcrm#Y&kyq1F+HjQ=~SB0oI0jED!k86I{#ri=u+Vix;CZ%@F;eI!$O_e)2l=pIA&lNogfOhOUuN6y#z+M4U$yFmmuKQwSZi2CO6 zOB6l^a97e)f8O8E=lB2f@po%)HGze*ALkmdX+L4y$(&;XjYb}wq63l9(caVg)yLm+ z{qEgZz|uIKXe+HW6Xu#>t}%vO26HK%* zm1g_oPB4H2uNWH8f)z2C*N9(jr`bO+Y0gl@I+j^x_KG^udcn5|`0bswn_CiZ81m{i z(xZ!+7Kv!KM|=$G+Z0C!FNo|06-sEwogVC8lctXj?wYAGn>lh-XD{j%qgfdUncs4; zhHioBby%1Adozp8EAC9F)7oZf9LW4S{u-3spH3XANF@1I{d83GBFEzZV<}B(djcUR zcS@SyTds8{hyWuVaq&~%2FWKN6V#0Kwn=T+>Vms)>E9z zklNA2PU5&^z4Y_4n@wq!EIC;#fPd3yy3FU9Gz~v8WdoYF>(bEN_)9JfX^C6dC|~1g zABuuvh4?s|-Q6nT5O0XoF$^J+bHOU0){J??O`8c7S6LuHkFxruzvdI07w%G#HNr)7 z(Flx?5;RS6$IgpsbDbxr=G}|AfAAy+W>lzhdX*S!-l;oQY}XeV*eMClGf%f01Y3Q0 zA{qGWCe~cIu=^%#yJ3XzgC6~j2e31U* zZZ{@wCHz&DwEwf%ENoNw?MWR*>ysso!R zA!On0SIl5md!a88qr!Itctd^K=`jb^&qK)Q+OKGV|8yd3t&KD?tFL>*1Am!}XYu^O z&HQLIesy^;xl60fP%nMN{v^Mu4l=hnyd)wr1nnH&%qH~?HZIi3Dgbtv7 z)EsMgT_wFQf@S;QS@ryff6KLaNKVj{(jKCqns-}W8+`lCkA8J4KY#!K?(>(6Vjl3N zf=5^aakyT~oMvpRm#4Z(^STcpYE$v5rOKP42~^^B`ANF8>U^ZQ;zWx_+Q>jE^X7cx zk77LrUPm&%jS|7<_n@TxdMfIYJba~H9=69_o7ErdD8I@k`Lb$|!U-M4;)%W_&Fd?t zgkeRGwh7!(Y8q*yg`u-xjz>D$?P1INhwOMjtYj;>EFkGeE-e0XUtJE2WVFqNaEr&6 zw3r9vSZtpLfHC(zcW3B7?sJljJky>yj66A03|?CH!!IIiVj^X2hIGZvJ5Fa?67e0I zse5IXvfuT9s*|j|v^vLx{h!SVAp2G$)TALf?|Rs(Q@*+5>Nv?tGo>@D8FuEw_gh&p zxtzx~y1{w4k7()#wuH-rb{xLL|JmJt-Pt!)gNr)Pu+gXP)R7aEaD^cOvYwAqHM6eF zmrqgy->n#K^p)Kl#uKGi(?v7m1xLWH(@C7u!Y)H*8sg3ax3V2S-z_4B&lnT&ba zI7>1}rDUE-EvWs5(QMFIM|#PPu(8}@j3r~>yqOkLtZ?DR8{GMrYTVy)%fIWzf5Oa9 zrPT=X9zAd9y-_<{svz16b10fEVQ4~ly4%DKy-TPAwCMU(JXe?4nhMH2E<2SbBvb&w zV{68X_g(@4(g&{dMW!iZ7SvD`>qrSb;By!J(#)eXl&i2RQ|PzbpEHFuX8VuBE%&0N zA6~Qf>%Mu-xt|Zs(?}lOi~8sM zj}rDb_F&3;6JwgREy`3p2^(@it^t}*jFZ`D(6|8tKHaeSTw%Sgq9V*=EKn>=zTCB_ zRAQ127{c{^9n-Bq$SMSgCto!g645Ic!{TeoQvI{NCX5BLWS*$Zkhial2{+fQAB3ba zD|92;g+or>SYYz59S7M)j~}b1gBWH;4v;}Yqznq*n;oGKfmfQnkUOFQ4y<~;4FZ-1 z(|M`jY8*>a#DT@B10_p$q$K2~jW0{NRn1GCK(qwAX&psCGbL$i6CB<^>1z@gW?QZ< zPW^uQ$80z-rinVw4#ez^X&wcOG?8n36|+oQ@VQs`6+cu{LWM`6gca}+tQX31L1FOu z*bAhD(`egVJ}VzBL|O;rwdrDCr^U%RZ>$%(CB*OVw<3Vl7K==0Lz!`O$=R9@-YfP* zCPvjBr(Zd9#e?sT_Y#tnX->_+5U=(Kbrw z$yYGPppK@;u;s^tuIE9q>?_%)KxOw_K+oji_%2V+{zutjg-X$|N1GKp z1Wzr~SeIKsl4J9;l2YR>6pIe0xb>`Gc2i49LteMuxpvb5bvB$^){q-6_=1K~SkV$R zq#6M}Z^u9LdN$rwOCc38(otOHxCK@e%945f>?kvd6)@3`~po-&QXjzp9kkM(Ze3YWrz^h-bYhC1&Jm9#r z;`Bm)r_UANhHzIXhZE23#*GjzWmD!)D;(+&*tCM7&|~^Tod3i9Dx>U`9411DYv^sc zAh3G?Df$O;pNb$wrc#6vQU(F;o`^DROH{j2qlm`YgBd0BCD;uW>(Gkr6=Oz^4;r+D zY`(_rsEFE{^^;%b7VQi>IM4c`o-CwY!Qf<-#W(ZGz@7%Ih<`eA7$u;oFuxad3nZXB zB24bT4hPIke9_W1>Wc5qp|4-gkEUGHJg^8(+FYQwe7uKhKILVaI83qVbDiY)T)}WS z|6GOKN*&5&;8q=*>}AiT`V1`cmT7vlu4`ViXnzAEN-@-z&GROQ$5<-Kb^ssnRG{ZU z#sdj}1X2Rv2Couuay$b~jz7tBX5{zb;;OCo6a*K|N*G1fAG+{N_# z7O(26*Xe0QX}^(FG4`{Ch}8NSoG1gr+I3x<3Q=BP(p;OM6IZTYoxqQ`c37MW2eMTQ zQ~ss>y1SR>?V6NsNyNZB$!>a*vaQiQ5%r`s!sSF~_in_WT32zEI67(c`IA?J+f12U zqOP=>grubL&M`R0MEugq(s(u7cH{u8od%m2;}uyL;iSCUTFS%G1>~`QvnQHtRY^|X zpV#f8O^#O|g#OS>e*IcIhRLYr*FVH^+OK`zZu|{1qwS0kE|m0=&$>su+rM_Ib}TFI zu_Cix-y|cb&FO7O;U)SWimEwjwlUBf4}_8W%<=BkZW5 z6LlSY1^_jS*eUWEoAbj(%Wxp9EUQ&$>xq@6G_Ix+dl3{wU9?)9VzeI6KkcvtC9M*O zz)k=r_-$`nH_eLF7U?duMv9sk5~-npPC_~>s#li>F$(BH&i2_wH`n6NL%umI?Ue^~ z6H%69L|q{i3T1eopt0HVesSu^+istQDY}XrTPcaFpPf!ixz*x$SP(6D-1H72po@}} z=!8Em-|eQU%{hAWN2G3+*gARWYy|rv!Y_N1cYfy8UJloV64eR%+xW?qtUiBIR(N$7 z*Mv`ZDJzla5Fte5fSxyHtT|iV&X8@|j_T$<mPi_)#t7k#$X z@ua1moCd_>z&*SUU5*EQ5<0B$eHjR^CYL%p#D6jSA~TAfhc4w9surdd6{x|TxDMrn zY_v2sDj>B5ItdbXeJ<+iCZiqClFGaN+RJmY1!+<03O;I_RP*;#BP=mJcE@44(8fUlCz2z!4_1=!P(cP+;!L;#vX?%vAe2~HNMGL_3`X`(OY`JMz zO#<}CpF&9{p~Q_d3qsnh+xS4UOo7p$5fC2O3(gx!x)}!Mz0y>ti|kuaeRCU0|H{~u z4?0&kZSfX7$9+L^$<`dLrq5k#rhlt9%2!1)eO>zkWaD;YixjfYj_&STWzj;L(#1r) zzK$0-V#$Lr%o`gU+bJ$Nn;BAS(UbXsNFxK5hsH>9E5MTnw!@Cp zUbAq*WQlmbk_trsofF1{o8IvAxElNJ%ibk%JHU|m9NZ=UJ_%%zb{1(K+|^X|d$YMJ zh0i8X7-@c|tGP1i2Th#jr`MLYvP;Vj77%?h&&)IYB@Ch#{-IQB|^!hjkjvj&_# zV$vro0H4k6&B}{E1x$8h7fucf$xS-!R4ArUCn4J(H6_Dy&od9~I5edh`}1xKVIN>B zJI5bx(If}4N#W&520kUjbZC)QE8j9|z`bcD$<$;-dqRLt~Xo0?A_u#*AIykJI0dRyYvWy_^k_KjRUS3 z4I^@!CB(9}C9N@5R@P6Q;NcbitKq!*e81a^$@1{X^6hozdS;taD6y5=>}pSf{X{QHJz}$ivminRA|tP!s(HFqYAKXQv+$D4 z;-XbbtDbvEl*5ZU7+vGyGa-`=Yx%Tzx+Oe#l_sf)0(Onw+AcWJ2M14$KZn@hc zEH&Zc);>hyN~b^Ckk#iikiJjLfVz_=JJgGN;XV6aU{{5S*b-7gZA0?@W>WST zwPbfj2~+3B%8W+Y{LoWq{Kv8DX;mJ**~a#3!4wUf#GU7Ws}tqiFWqTllS zrxxF5-9I8Y(vf4__PSvFAz8&80MkTL8qcK3o+%h|_SThM>6`9>4{yj2>)!Z}mv7+G z1BQFp1w0x?l!UUtkLCQkK8cJ8G}nwC1o3LicL$(_Y5mz@xes{tgh1^IJVBVY*LjZ3 zFooYQjv(!RKNI$|uaknuhS1ekDd|scfXy&W3;RWd_mi9W@6BRySSL0u_=yTtV=*K6 zM@?3F?F)c{1FOU1+3cpB#t^0L%MfWR_xFQ4o2{J2C?zlTi7k(Gg|ZQC*Nr)^2I_v? zc{D5hcAj#X#ca5a%}uHHEr2xK2GGvNMb3cq;o^nIxcZ40kDKX*_`5}f@~D+X#`gJ) z@*()}V6yD*o4dRs2cu7Kgs}Yl6{_+5-3%<-U(mPq?&*adjl1_-s||ZR*R@$e8@IV; zU13N1mGo1p4S&GbrIU8fxEVzSD@LIVn_QvP+aALX_J^y^D?DnD^aUMGiaI2Hu@lEimk=Fl;^99hkrg^)y zxG|#*X+*HR(xxgX5rd=J4S9Vms2Dj=5p|sYCg1@q|05oNjV+tl>a||u$-!~I#)1N+ zsl~ZD*W(nbOrxd>JIYw*!H9YWLVvibp~ zpSCpqNPjc+zVloig-m8S5A>5($v8@RAF%XvP3|arxABI*wv?)S#fX3fcwb^E!AOk? zQ17#weZ2G(4L(?tMcKj*GCZ{~EXl^f*8llp(Ka(YYN!SHv>-1a#a{ebI<#nwu4(6- zyu|SHmX_L2MGus9l!2Vr+KO1urHSU4{;uL&`bqtJX5Y9$|Aj}ii&^5fo}}jtHTGQ} zQquQ&$|eQN8~{7rm@g>65#Ev+=O~IraA%Y&Y;O}>SFBP2#KowwQ^oi)mO1hUsMyP+ zbdr*RYJRYr1#J5=fYGCLb~&oGF~tEIoNyy>)>Cpx#8;Nqzv!}5v`VnNHWpTrV;Ecu&Cxb2w0`%1X-{Nl0*cfjKQruuQKta!8P35KnHKy%NzG@px;k1DG zL%EQ?yVu7Ad}_w`fE3N;=>pLwj)#!Jgp+vHgh78?yz6yL^Hx2iAbA~^{|T>yw~w4a zg7?IRE?$+8kQiTJr+HE2px@-UT1B(%>pnBeoxDVv(+p4;M6~DL)m!%AL5T})$lOB~PDC#5+OHrU26K^)E@rB%c{AcGhDXIhOpe2G*4jE-ei z7*djG86~|@wZ698j|=nMk}$$~xTGKyqhXiJlFBSHxp!Xax4u|viPww|Av~^*|NA)b zx8T}tyx0{-J#Tgl2=@8s-UI?M%Qt5LHv*BTejJi_a|Vm~y*YtVizv9k1Q9tuV}eor zLDPlDtWC5~jQzn%XZLfeuHidz@a@)YgIc^v_=C?;={6y_^+{k!;LoM@t@tb4>;%^l zgpch3Nk&9f@&(nCAj*-$&VnFsHI;XnyL;r1{XPv=XMiOwrqYB*2MClMIXeNMQ_%b7 zWMIY90JW>vE=Krppr0Z}_!0XvO5}}K z${w`Bg(>VVZ1p29tsUrRu;t5x9vq-Sp_QyNAdfpP@qTgy1&`FHa%>%grlW1rDs0@N zZ4^#~D465#mz$9Mv-xf5+Gn+gK4PFqtsEsF@{^%ueOlJH{rQ}*1~VjPj2^E7rs{<_ zHbJXmqCN7jq=&v54kF)S}KF2H>F>v1yc>)SzNc(MrutdVT2v8-tUt0 zhDSmVtJRM1|F;z139eL{Z#YQ^@9}lYyZ(B`}zrM?FJRRl(G!$0Gq^M+BHiH z>;f#pb?RG3gpDMdJD;>q6#8tQfLzUQ*R=v$1?fyFjfgm2t(~bhvom=m`>6fjJj%$G zHVh~+;)mtKgM>Zd5sT+vjG2MK47k&>K7WkFbrsm32HB4ucf zZ||tFY=1#Qe>_Kj_*ffB`}-?43>*c1z%5$si^;MCZc*@gB-{HWSDz#*jIdA5<{n5h zw}RCk)P=LkM=&DaOj3cYLr4zWYj%&?zj>-RRuqBYN6>)BDc|AmPRt)YvajbcX+bR6 zAe`W)=LGDEok_C{DFAO$~0$AGI>`f4O@pMd_G`TF&v^wDAi0)%cCROx=2zMrP= zJB!Ot)A!T#{WN_)UQzzXy*`lRr_8ufiR-ZdyT0PQYP%Nv60~S%bB8|X?$D$)QP@p z*Y~{&)dQ+rRP%cq9^{{(_5$iL+^7*iS}dPQ3aH0Oom@B>jxb57GW;7SqIbRgUKi@t zA|P;8A|Fk(2m{>I_@L$zpdgz!!yF0Ja=X%?M9Z*hf$cyoH*Tc!8zd}U84I?$RyPlZ zd`pnIt3$j?l-7f?#W(-!!VZ3t6UbT?teJR@n#T zOqed?iP2w_&lx{pf>1Raeg{-{kiBTLyuh+;(%`eg0;*<5cGUJmA}Q&>&`6Z6F;GcM{lM z@6w*!Tk9(+h}IY}jUXncr#w1wNF7pC9q@zQFi(MR0e)|X8ckVxBO$x|7_v+G+R{E$ z+q+bo{iyHMMcnQT1iJLBMEX9<_Nr|sln&8X(;woC{?FPPr1w)W=~0_pdoVPwYAcyL zJ%mt(+rl?;dxhy(mt7{OV&c)4v3+xd5f5?wQt*803-j4&<680WkWUf{h|S7!qd}=R zi=+b-Pp@W~w@S(cQ+Hr{oh8q3Rf;jZH*ZWh#viMU)XtC4lZ50HX@Y8#zBxRZ;mkC! zmE{utVZ`OhrI5DvQZcMAUV_XWqiNLWvs>ejI}Y|&Y&D=8C^cP^|H*m_V zGgDLgQSgEzCokR!lD(fRMHtBGTuirlSaE;a@$+y+1(0Xj_Gg9as@ePOVU<(FX%sOU*7HQs9p_I z+Hn%%x;|rbijxJvuQp2L#(lf>LoUH)nq3;0EK*OyJ?}?MuhlWza7DN;*?*}#_A4^H z!NyrKSM2+t)S}qkrV;B0*JAjeoQeTd^MMzwa@*?gbMon-1w-qofHnM3r&$TEJR(5C zK6StWr%;4XmZwEn|EBeTUmL3k5o(bT|6e5pKVr6R9$i z(_y8qFpIO1PZGz`4Rr}qh_};uyzqrEn`nV6Qp=9M{9h$-FL`fNywYc=Gp4UyDTH5j#;LD|j5PUs* zUHv4XM+e9=D*_bRb&t}S+TxjV9a7SZIHRv0-iCG;`ClD@0DHUjjvWy!?Y#4tv`z0g6YAFNc9J(c_&B5dg^{JZ9tqR zfuw-L34B!gnmObh8Nfp|pGcuZ)WUoum=Uj{FQ_ws{1*>9iGr?*sHE~q(T~whTS`4K zPt??d@zim5^umsGDr6NeXa;D!%h?5R?hHp^lT!e1MiQ5s!Xi0;Rx8WE2kBgQ2tOX; zWqE$vSGG{?NqSijV0?0P3EmevU1My@eUP$59pKv~(wmo}WOvx8;xh)%10HAKRRvi) zrjTYY+yN9g5OK*}auiuNnQI{j3vA_v*fFs^T63-uuQhYuKiYq6@&sz}*gd)CqffJ| zwknVL)&m+W4dcNpRzmQ(q70VxF!r}zJKHO!L!NLyAbXY-HLR-=f1;jJ<|Em$HUmbO z<*|FG5Mu&IkMV{-LeuXh$AdwC=fv)Q=uzo0dFTxQi4+@xsT8ff#@(*)+%j4mg+xi> z@;XW|fY8ZVKnTau$=!QJEEK1^UN*hyLpj;>-se~1Go3%+3-Nik)s*TVstY)3?7h*z z8OK-c>!nRvr@4k};x<>Kd@}f-$8U~rOMFh>iRnzq*u1|&gD31vS-Vd3nvO4(;Lb5y z%D`zP>tx0q(=h>6q1x&9h62!6Ep>+POrCZ6 zpbbi|!~*o(@588sG%;vaVzttXNp&Gjd?gKqhL8_*9#z*vjJi_wJ0@rIXXdkaJoUp} z%4P;Tv#$FOz0hxQyTn&9+P@-RRz?~LHtsw_=%niwM3FJ!ctkC*k9)vdVS=1k%5r1K z_?{=2I#iYPO^Kvg6Js<(|5pi=ed0%U&X&`vzrAN9(mmZ znp>h#-qgKf1+PjecSwRvJ2*802{x~NhIL}su`UWj5(4GNu@O6G4C_EZy$8|2m(}?N zEn4Y$Zw_0iqG+$e6Dqid)9vn^vKc>4M3|Cwf%h!o?J>c7m6g+LrLGRx!bQ0z*&U*Z zM^f49fJV6>eqkM%ZtAsNsf2Az;Y(|6_&*~vfV4oV3k4w;$VsJN*M9R4QH(YfE7n>7 z2TXz}uH~<{?j#qxZPFkb0vHP#NlGj8)*oM9wkJ8g+!>o{Hv(mHWuS1o?pWEd^qrDU zGV0AMR_9l80KqVO>}@S|B0T{W?|D~r2$nNlP{^-WoQt2ECrfFb_1frf3L{@@=2n~~ z8_z<6!h~97@`}4qGDT%bd%RZI#5x`RgOCy+?L;|F%Ql?9K2RH>kUl$Nn)D$KFxPR{ zETLmryx#k)(IB{K* z6(#eoK#UeOhn=x-DqTM_T&|1EN2Fzh*-iVqZvX;7GWq4|bxHjAxu#mPjh=OFMpSuy ze@h@Bb2Py!o8-w2kBCmMTk_mmSK}pB;XsnzQ`Y0-;rVV`!HCTQh0Q)9X|Z^&qQtDv zIZS~Gq`*r7$6O5a}0qzoXc4=~lL<<-J8!y$0;4L49xDZ1Go0^A7 ziIr~?;6*`%o8#dvKG`c>Y{5DF5ig(G19NGe>U&_yHy!Rk<2>M-}peloUwd{K52 z+NFb&wjVu&Y9g^PN;%roLe5-Xy?Z#<&zX`W- zH7{!IbTAFXrwBK*0cvdPp_Ke<5E0xnZvHShE=G$jUSMu}dZvspn6|$3#m@vFX*mu> z386)*r4?T{*QycRf#<>fFyS-F5rT_7l&9DW79U(wGizfagogF@f-js zp;8whtpN2T@@JFi37`hE^pooLYZjr^ryU6e!)`7s%mWt9Q%Q%y9(_%e+5dnj3(ZIq zH>y8M+zw?n2AG6*En208D%)#lG`1wdOP-!*wtuHgz_%i(`)>o7QlOdh3Kf+JW*#e} zyROo@&Laj8kn2yU_(#a=yCQ!mtJTGLo)Q~$FZ&k}HRQh)QF}L=xn*k9lj)Q~wZN^B z9x7k%6eqRdAMb={a^C_3?)d#998HD{KT?w%@R_BZcU3LvjewTim~GI%DUlaA3O2`I z#jmBtjML;;i@beCf+~GU&;g02}V`Kf_I<&pcfBgVZhnTU+eDonk ztRT(FjJKtt^3yl;_J09#i;$Echc*CYuPiLjzcfDD>#4rWpi2{JM*^seg$%g$2l{1~ zyS5o;f98m38eb~squFv(Ul}5CG=QpTqp}<#l!Ugn?hNeAUywsEae8^qD+zA@VmLh} ztzY=S7FRw&iU0w`uraFDT^$;`^o&Ks+=$R0l_9!Tsv}TQmr?#?z`04U)8;ap(CS53BDO+RZ)-AK zr>Q7}FF6R8gR>@8lTWXG*@!~j7hM@$29!Iqz7|-sUpyD$P;tGilbqns2b4UoV{Qn# zeYj7OaO~Wq@pvF#gekSrC>MXHvdWfa>coSq0B`<%b=7YL3(ZIYs5rtrCXOD+JLKK; z|EKInL}dDn)wF{3%v9Z?yW8vC5yrKRSdvm$D5(x>BTjNWIfXxEgZW!owKHC>#B;s2 z-c)Z>l(+pjcTUklnDyghR$c?oBG{LOLw~^Kn#e{L+LPOxoazN{!9enQ8a~HF`V@n% zqqVxFK|pnU<^9!t2$RlphtY=ne0u?mTP@-|3*r!q3IDToGZZ@p2FASh^+!tBgF36DmqT~bY?@l%#juolJ@ok^wgYxz9} z+~O|yUQ%_y%HrpOVI3kpk-E18&x_CV;!rGoIn*&E($yKc^D)D;!F5t8Rk2~KVWhRz zaDoHW{H~Gr1fmG`VW1{~QLV^lRDb~y6$2Aa;3~KyH8CpSK#MR@#nL}R>q?nekG2IQ z4_w(brvn8PVqqj7aDVY}m4AZa&IR;xE(3zC8^)x%%i$fF4AG7S)9!+ziXAfMdU^h` zkG2p=8}|k89c8_35}bCT`hSzIy0#49RoA5V1Ij;_Y_6UK_3!DdegRwFQ$`gGvm{fc zksoa%TNb@GnniU-n#WI$=41a+KeXtjmCUoCul`^+Tj49l8him*!BNk1tzH72wiVfo z-F>9nFT;rskokvF(Z+Vu$2?|QHiaU^d8icZXIPH21 z=|=zY2@ILPH+k}$-L%%jHb6r;@YxJh)VD&Csw89aLAM^Eaj_@%$spj+&ZNV!uvtj; zyoD5y+AOn|az@=Jh=AsbXQ1MslJ?9ECWP~#Q%32ohp}yq$>Rs^ZzlMkfKmg=2?rJI z*VvzH_U`*@snErfWPoWh%zWTQ1Kfc4xJ$%$>cDmZ=;pJf=AXJ6-X~F;HlINLs2RlR z5B$FF&)@&~{C|$WpY!i8Wi9`o+n)!qmE?EZ-sm(D!y{xEi+4)JjkODnlq>USm zF<%@|n7Oez|LI=~tA4Gqiuy%Eu}?w^+&PjO^;qRc%|#=C#(gAD?X~|nw@xKHXzbKc zG#~iiRI#Mdz7r`_Dltec#c$#iZG6dH+Sh*pZpG{W#jOBDPjMA&8das&daj~m`Cp6g zeyzjf7*Pw!htNZuz!r%Z)gLsSrL=?#xQSXmsX~&=i|93lu{_d%Uy@0Vd&vLR9Xal ziQuYHFCM#+Rmxzz{h?{hfC&0luI4utoAvY1-(=qYMYrc`Rk5uO;JYzw^AHvB&F|~; z{0%uDz8oNoA6p)#nR;NKv5l?lN@&h_NWYc-xk5l}-<%BIN=}G%=~D}0b-BBuGiXze zb<-+$_i?|U;bA=2C|0yHAx6_6J{i}^Ne1+2hW3dPXjgDXLQ1WZ5pe zKOJ8Gp(L@1kf<9+(#x6v^s+z#0VD;pf2BX|eY!Oj()%bN8d*=VfX1LekvSV!ul7th zPzhf-itft$QO{_hR%8(o%ph(48fg6}ZGC-?wy56HXYZsB<;`Wy7)WH8_xL|^_Ol+@?T z<>%zJP5`0#TvL%kuPuwo202;00H9tFQNJZyY}z^Am_mw2|5=Xk10b^~WpJ|kTdm9A z9su@m#*g1{gW5*rXD<5xkG=PfYHHishwU6hKtZJk6#Wvny zlMC6(5AO`J@X#=!h1Y+990zvolCHDp)8c+_7u7HPVqy160`4&Hp9KwTCpx{2fu~yi z(z!E#ZyoXfSUVM975%Nm% z?i8TeS94(1UNVQGlwl{E0<3@*J`CCb3oAfXdw(gUZMWrBXRGX;;Epf-r?S&lWD4gc zf*b#Y6T=@01*C&@x!kJp{pF4N2^>oIIN9RwThYP3h4wMp4N#5k^joq5icd}Ubsa4^b z)9_fIB?gS6yK!;^aN@XSgH&>ws8#1u$wGFrT3>e8o&~_KR2Hh4by)4j!?Dk!`mnCratG7-J|;c+-SOj7U3#( z5n}=lf`k|X^-ybpx_Zwu^9+@bz-?W6o^U*KH@8!kXIOIa+Sas4irSbeo>|IbY9-$i zakVPAk>^@J?@fP_DY}_qELqA_H;9&lJM^Mpl#ns~er`$S!2HP4+^XM%Nn2RGX`6zl zUuQ)TZ>)DW&CJwo7ER-#gb}Kg>~)u^KfIn}F)h7nTgv;tY09~yr1gd6nFsSR{a;cVOw1y;cmE2eG(JBt- zzGu0`>6e1#{!)pL8nQ=`RvWo=O?(plUi;x{YK^=)o&fv%Pwz~JYSqi7q2oB9L&Fo! z-@5ka=5$TY+`cEovKRTxy!VY_RuXFiXp@7DlP^Vu7I z!IF+6oi7e5x0iU+yvxu@o5_u7xJFfl`K%d+c}MB+()zWE8Sxag6g5{?8~BtX(pEaY zDt=l5cPh7%Gp9eXrT#>YtCU{-zI1-~RUEf;cx}_-3ewh{$5pt=c8oSId;dc8B&K2i zs2gsW=}ooV?n5O`EN=|kER%^#;&QbT>4Z=8N0ZF^w1+UaD_$A5V`DG-yy*p{Z; z%60f^c_DhwJt&M--soU^Mr+uDC_h&R5#Hoo&=|fmbDwun*O;1Ntnpx1P3G_`D5UGS zbCt%3Lqp)0JMk%OaX}?yJ!B>wKPcxhGGUtL)tTtl-o#21dW2ad+Eu`hGw9m1$o=y=j2;qmDIO^fX2t*U(X{gG2W>@`#hAVqNWuA&wre%`-F zswYMR9V_)3NH{i``MSOk^j|KY?sg>;xs}a(}VCKEy2iFf}Jq7%fhRe<({`)sY zQSwofXDYH!7WBg`$4G*xB9xh&d|u>09rPuq1zXQ_0NN?ShSC6uu669 zs?mz9nBNm;LpVKa;{uIen!GQ}jPQIO@Pe)WDR}xj?J=hffBb7?72|f}7H`KEJpC%Y z(copAab)>`ff}|um5t>4*p5m^;7~hGVAuGfpn*W>MInLfK}I~&hGh}t#C zx;cx=l$Tm58kEXp1wvj6>p3 zbza|ytl`PxrYvBUN`cn9b=A@6dSY0V#K5zwPgV=k#s<2TK!VB61NU(tuiCxcC3H(R zwreof?2b~$@^I+~616eI+!Ni?k&|iOXj%8g_C)#a`1-^0!o59|79edj9xWAIAhvy{ zxTWAMqFDRl$Wt)jDU2_JEOG?OngCHwhT7ZUHckC!0eI#&+#Z3RVW}nGP5}a!YRjkb z%yL%mX$ve7V@>iK+_%iAe{rFL%Krq`#=guvHfplQQU*oEjv{zSx!qlnwa-0uL+4x# z9KP-%k)IR@=1MKL#neljoH@JbP!2%8l_$^7J7rM}dO0JS8i&O#5zqMCOp-8EQaIcL z*xkL5E5@63%h$;QUAg_W3vHu-{?usutY71GsWZ!r)qV}sI4Qb1V@bRE$PH=2RUm0$ zQtZBHSQgDz(AZxK`F3TYck^5OP}8YXemAdR)WB_{rr*54r>;)dw^-EepUg*E{NFy_ zrTCJy>reIVj$N%mf{2rcc4JWZOf1i>7|w!xmw-WG-rP}(TTU? zTx!jmkj(?qs%pL0<_F{2{0|;0vZAnclv{K7{^y{)t&}EHS``7BHD)!(e%5fLowB+9 z3#6W1?!Ys!&K(BSF>u=>!}9wx#jH_!7bivHMm|fhB7ha5c?oOz^3#FP49{+5ru-@^7Dv=pDi|QpqZKL z#%lP4|KLCw#aZHFnJrt!Vtf6+90o~V!4Wr6v--}mrO;PU1v(?8^o;Wl5p zzR@QB@-5Tom$#E?H07eB!Hiys)E&h#p6GK%i7P=!QN}0>$XWtdmFOrf!iwiA5<4lP z;bf{OfCY%2>;RXK$xyXP2yhMJJoocpQ1`S5N^b(%+6@OI6 zg1?5~FDv-#LHNrT{N)b+S`q$Q6#t3~{=W^4zFe)+O5godw=-7ny81@ksrNA!opynO zX99_}SWh{b5&HK$_K(Ts{ja(IDv$j`aygo+ceP4xGOB6y#oa=hKjhB;oXY;5>spLa z8;LBd30M0_Q{!Bj7H*PMfDFI=f9FD&+N}V^E%{w#o1KxxdDUQGN>QsDb`ocH?ByBrffN8^Rx}|R_1M9fig8{?CNQM1^PO;nAahAPAme%*DKr>JI zAZ+-A#E0y5lOl9+qLt?m|H%RVVJ>Q1qABN(ZM@+5{QJXtlhf3ZD`rLXie z=VS9Vs{aCC{v#S)NG?%R6&{Pr;sI{V>v&)a<;J`^@#Am${`C5xGA1)C!g-t0w3!U6 zqA38pt+(XLdY>C(B^_SbSTryr9;RA{a$$A$%d(TF#=+iBSVnivVv}((>3%yZnsW8< z&dBa%*RqZr`V*xAI>)4eNW=GLt$9n5g zlf~GGBbsW+2E9mE1A&q(IiWwkabi!Y7~A7?1JZm;TH8e?=gyBWkygeNTig1a(jPJ$ zM{m3rgoxxj=&x+{;@WrUx;MfF~t zc5JlTZzhZA(&Ep5Ilb|w*pROV!MiTeW@{d}Hj98kFdUaLQ|KR61ni3b`Q+#EnOwUpc zYV^l9W75(R)Xjo*e!~jcFlro+i1cXi3cgg{Nl$*XB9MdYTJG<|Otpft(MMa%_Z2v7 z<&SJRY=wMglnx&;X?J57BfNuk4>NTrtPta;xPsbWh0y1)e}WO*t%klm6t%!Um)VK2 z(5~QCTo|-@=TN^{A&Q&&zW(Lo)Is*4z(j6^TKV$EcGqbPQGsCM-wuDSg&2%?3;2W%P@lrahO( zuEjWZXlcsNsmshxMu_^f3HMg6#)`U+5Id4f4d!2^UC1see1$C0jd^k$pYY$mp*ULo z`nw-L+ytPejkn}mx(gJ*Oul?6=hie9Qxz?P-v>$nOw?7wyak_dFyQvaxh72z%>Qs} z8`qcsD0}|)iXZ7j#{xowRk#^f&p)SZUsxY8Cgl*ex7IAa;5rb(HyV{L4z#8&)UCuV z`r^q&uS|r(S`^kfau{OGts8DM{|fYf=cD{f0t zyi68-NOrC^G4Y*iwcE_S=q>(9qyI|Uog>H2zBqE?(vg?FjCvKf?G)r|H&))AJp1D2 zy|a4NbwTJubJUxk{Hd?V-#Plr&389WC>Yc|5IUSP)GvEhEMQPobb~5fT2+V%YRR!TR2WVAm0~rwf7CMa`GVK8&iA`U@cO;|)lpPJF%dOHwW>87hf_(Fb)Q9p z=24i#5nbPdIbI<2!7Y3yjnc_m9tUjwVVl?EYtF~)^B!^nEuydG4or#gt6P~;BdiLq z<(`@b-rUQ7+(-Iq60I0NcOmoEe!taJ1~-f&AXQE42qFvm;Pu6rS|ls~U7cjp%CpAq zyUd%5gYm5x8$A%7=LOP}L}S3ogy7E#QV!p}RZ3HlS6oun(Q?xMSjojWWoYb0 zK+PXp3eO<$kc$K6;Gl;grYxYDd4arE&oh|}jiTVT9(}z~Jo9~SUn|eB^wOe>Ga_NC zd!1{{iZ)qxu7I)3xy&SZ&o>^&gvGviC8w5BGb8$wTJtyqPQZ0B0`-npnpU~0yd~FA zLBrXFT=c6NKiz>AqhAIFh&~kk?elCcO6o`vAYNwV@yzf1T zXEud)2qG;+!?z1&L>h2Wh7m|hR=hFN!h4*%k{!_@K3yAu+^$h`k^x2Awm>-$c=z}* zH!x4%>s49M%#vp9T(x>j{2hOI17YH*O`Gb?uc_RcP`ckRHcO#Wct(UgB0iM{~!MfQE zxz!qpr?DQxl|>ygbZY7+DXZF7)Z%<$m3O^7zs=#+eu z?LW*&Wt`_f-p-#sPr!8E%xFRF(z3Q^1k$LDo&Gwnb_B7*Ab{A%>w!S)jwe9`Zaolb zrWg#K9X$=ksuh8^kQOk{_R0&M==i6wWdw44?X5HaT7q!X`Vin>rY=wDOp2KL$Cmn$ zFEnm44I$X3s<6=sF-?ifiT5@L?DPH^JKrEJs3p8pb~+ag!f+)Zf{G_MG9i@)@)!74%{CB_q zhnFvZIR;p4Z49slH_-e~zwz6bZ(e(8gzXPF9tq6kQ|dqbhyU^N^`*C{yI^YExf^^z zzpWsDa^j=&)U%?6pqrPQIV$M z_iK`!Y7)Ko@WJb=08xsDN$^ZQ+moTXof@~H_o0V4(TL8Bd!ZShRmmP1F;0xi zuf%Ak`AymGZ1f?xmUMQ5b~l$$O;j8_;qpodO;%wt?V_6@G^miphKj@UV_t;;N8-lzUHzKp zw&kggJGyc&be}>Z1n$-qNmd!f0qT4`ybtpn+Ae}JuicNAavV|L8k;d)C)Mq&`x9jZ z$1?#p=ym{$Z`U>T;X{P^ePgTj%g#+J>9PioG;pd?KYp)O!h#@&uFK^%@L|sxjk=zh zbkB4S)2M2U4J4#>oC(w?o$t4BbTaHZiqHBQnZ>ny}ta1ffbtshSV|#}>Q7X{A;=N8WAB>b> zg)3+Oon(m5<>`CnnGBt!u6BfK-ZzO|^PVa9AEFCVVC&1kQl}$16C*>AX1O4|dSibg zK2Z@7ua&&?p^N?A`IDo-UkT~!JahQVNMp#S@4`0pU3;TV=x7iNAz#i4=2(q{l_swTdA!7M;I{`Y|WuIN+3+K7x zcret+^y}qy`~JZytaSkCZw`?ErhcR6b)ujyqjI^Zx%&~&Jn{sByh*?|2b#NO6*!PD z75DPEQ!K!P)%szK^LcN!{rdfSLOgSN*Bv<%Nk_N$5nc8Q6ZmJIQ^jkAtfK@YHyVTT z5oM=Z_m8Fv0>V)4&XuLNoc*0~f zlsN*E&gwl(8*1Tifr0aAh2n-?3DYGaLwGrjS0L6y-4!$6vHK77>Qn_# z*5La-SZ0?yoItDVXfgBB&2~@p>oS8tbx_yr($iJ3`vms^jbSKmwRA48R4a1A?@&Gq zyi7gcf8%-}e`z)D!nBc%ShvGCQ^aYdULIg&sPpizUC}e6GbBB+??G=lYXn`?3?(HM`v|9xv0|&EZDR&&x~j~&YwO0c?8C!#v0wTk1qvM#6;_i6 zMylwSQdg=KCOpj?KYl22NEz|QQCrxMWI#D)=Szhx(!g*X*H4LE(&4)T%_%ULLHBIT z4q<)cV*mAD?d~xc_`C9}-Cd#B-5s*svBcSINo#FQtUeW?Udcp;ZA8DKTXK2jP+`lK zD76Ze!B%O8a5+?tZ@I1X4~D8Jr*HpkUSF``L>1I7W5+ujIUAR&a7vbY%^z8|<#1J@ z(-m^jm)qv>}LI@A1R&$vHGeui_sIUsmW#od8_u8`Hlz&`!XCK5=YXgZ~;?whr@D zjn+A#UlMFh`?qTL#;m=W>4>y!g7$^uM!wr-N$P6_k!*X8$7R-D-1%Svc3Cm{T??Ro zcYRAv`vWxH#MX6~V#AIW>?6;b`@Q>!5u>}*&u+E$wen&Pc9b??ezS#5V_ zq(^uNm!X8}JK+=6wT;m*VY`NJx)M#}L4J>)CqpdB&ZN12-2pt}tw23vDOa(^d|#x- zp-W;g=3w6`j#kQH>ToxDDwVIv=eKPXsnX9X0KYc#eQWyop=v z>WKHbUEryY+rkLHNTfD{2q{z;@;YOlni#y11hF7&9|k$0+h#PRFt{E(I!#{femIK| zDo!XmGeTB#z;xAlogl0TGkvoP@>+iPmSA@?Ls~6)$>egkyW|Xdz{m9$*T;h7Sjrw4 z$FSJj=)_^8#?vB_<=}X2c8X(lGtck8_`73GE|+1_aik@TJqd|HLXurA4A3c1m?!GT zX|_W2OPdURh0)TTI_DXM&o+|tySH0H?UcAl?#- zhc-S47GIJSu8r-P0lp(^E2ebr-WT96oHpW(3T*Eea;UWt1%xtpgXVrq3?J8qV%n$Z zccAa(4_Y$!ET2OXfQ=~Jwaaz2VReTlt2T2;o(S)bR^3jg!?8>QSjN{w9kkXBfCL+w ze&qYgHdpG}@pW0*D+S7**9JVEeubBG^wtp0s3$y}_i>D%N}#_2a9-bC#W zeN)D>;e3a2OS`7g*9vRQ+2|2ZsP8LEasI-Yyq$6ga+}ofnOhbU-e_!zPnmQ=DGDTY z$&{QW`)4=6KWiNR>Y-Tx56zX(;BdH47bONy0iq$zc$0deICzl4Tgj$z??@Q>V@S#F zVlE1Zs`gBuw(~TNTZ5A>j@EIiPBeE8C9Qa}yAA%L@3&9;Vemrx&lXrcQRJ*2cq?9B{3Zgg_XVCi!Mf%d` z3+a(ZWeV?S@U@=;%)DV1G{nA}6E zz%_f>KnjG(0n}qveMwDPn&cjNvtzeh8czWUoK;-tl`ZwQ8lB$y5w|`=<2b$@THB6# z@GlG=+9it4xgTD$N%4&+8?MVwkW1h>eC3|gEQi96YsGd%o5ry@Ezb{*`w;y)9;PBT z8Tx{GeJjAs=pOmQm@yC1yO_9>&c~t{%ewBhjHQ;^J$icMQxxAhIqnzDT9U(-V%0m@ za(7niE_%cHc2Gx}RQBGOFm(iT)~8>xh|+44->y2W11RJBSu$qC1^F*(-9|jR#y*Lr zI%v<3w#Pr;blGEosTF=>^nB{V`qrp(`|-@6nc~-l!a$Jmja&=r^udgecF8zThWqTx zS?OrF(6b?|{gMYY3RY-F^!e0A31gd>@-l#;KR1HWZI@eZNc6WQbd4wnNDRCsV&F(& zdcrfd9xOPmH{mqg9JLATuZjdVaX6O?53H2-9JqDsGwp=<@HF*+r3)7u6VOu2T}<@q z$1umaI1Az?otES#h@UI)8zHeaE$SeRT)>1=yn9&=Qa^OXYi0Lrx2~3T$uJPSgW{cfM!2@R=I76QR^cmrT5qtw0~CK2uEIt~1?0 zWranvxb^ZOKU}gODvESy7{0hZgXm+h5H>}GA$qwpp^vwN)~=eh_%_g{Ae{} za`D3M(zVfpVH2Wb!98@XrTzQ0O@}uBF6p@>9Z%h1_U~Ip%5(Af{s1)fc>U7 z7hN3ovZ4crnZx3&cI%v%6efuE75L6*J6prpj!(9!UGpGpo7z$|Y`u3~SZ70>9QK+= z3S(TwWULPAJeNaVEh+KGNdjGK(GwXY8pvGVkzs}_!B#6k;jgI}p+e^74_e4}`ky59 zf#%|FK|9Dzc%bz48fo6^B8|9xE>KjP$ye)j&Suh9!>x*8YdP|6mRUlNqK1t1+wHUL znI`pKz9pMliu;#xh|-s{3Sj08d>4g`E-7*N=#@Lh>bQKn#WA~~-4n-~o^%yTuTl@k z2+6Ht$6<$yF(2xvkl@AfTg7SY9^1bC4^9Fhp7)Q-2OL`g>4PE{;3mE=z4tnTAE7<_ zSs&jz8R*p&9F7F)?y6xZa_s#+cx6Vr)xy=pNOHb<_TmW)T3Dx!gc)^f-mKKA4S)}M zmiYhxM#F4{&exii?Hd#4jZ429^zOeB!lY&iztxZ$R|j@kO>>twuJ9h|POiLe)?P&O ztIe8FH)QNSLzQbJSBScNd8*ii8Vi}Y>Xk^m4V{3ci`!R3oT*!og{_pw@42MtW`10X z3FfXE-R&mUe|7|Oh~nQcU`0MtnWbx?KlEiXrfoK|Z%wQ9bO=^rCRd#x5iJ(y@yn<_ z?kZ;>+7nS+*+z1z?e)Y~(l_vmH^~F>KFRkQ3G+m=s|a5;>ICoPX4;IJPxmuWAWbta z481RnKQsOyTXSEN63h{o3bT%M4sy+ioV3x@`U*}uKJGK`vNJj%8FP*+xC6jucacsL z2S;Rx%QRNqW5d1q@cFxSHxMbhjGjMqz81hM=4CO3pBLPC9srm{5;e~3U2~hNUi~|P z1UNM|T>~}*>#@eQcsQJL{Irp^;y!^jXsJ`PH8Bokr7@d{u{W?y8A$>BFSE2|np~nP zEN?Uoo=6a)cTO}gTZ#qT=v@>9J~Z!RNZe^yEf3dCyNb$rVy2mCz_e8zT#dhGki~D4 zR`+~4^n#2-il~i=z=$l87Gifr%&(xiQSaf`g6Im!d9>Vll%^HC6aa6$I0L{NA?$Iw zCf^M`#$1_pmP-+HQK)V;_^U}oMktf;dUTg=p5D};V%#>wfjlC4?AkfBg%~7heV)P(=qrv)x_cZroQ!KZRu8+FsEZ-_` zg}qvE1W9N7v1?C{4C%^n&i;X@cgxwt8~CZ111aZ*f-`Kgs@CWfA>MSiyvDA*6QL@i z3*JB8^SO6LPv&5|jCT(KBp^#OwSRW2=}s$(9q6a7IIiU-Y+YwZ`}v*!?K*5`+YEWT z7Yc47XzIHYxUm4w(z<|Xg}Dx=%==&by*^{LwXE;!UxvRBHwgl=E>o%MGT~=7+=7ls zB-c#Z+mZ4mPtz;<_{KR^#i`yKVx|mon;W>0TJs=SS-1@ZRCv|-3CRxCzdSJ$2AO-Y zPk)Z!+g4@&?J4c*{(Zrn_1vDhX|--VN<+RoH|X{P$c*u9MFYX7(2lL}d0D5FTLSXq zzWukvjPNrDZXoYn@bTDbnHa~ihaj&_569;uSJC@kUpuurXv?GJd(Yo=<+B$R z(Hoh$LVCXp*OifIZnGH7j36jv7JAWJ;MN^MSejBSQ!v^BJw#%Ju1ob0fZR{~JOTa+ zCaxsjnC4>KnEc^kro_JKnwZt*Vt+g!B_CFVh zF=fn&o-lgu2^wVkUN^hvfqM-a+_Nn9`&xErorV4LM? z2N61B&<+OJ;^6p-c@qG7=(Fh88TOJOiBXxvn(rvmlm6~~(*f_xj1~WU?_oMTfemp* zC&>-lq1IfV@UWR2AGcmL*|P|ik$9M>D6Bv-zyC|E=1BL$8>Ev|!YYG$_x005uY8@-%<#2Vw zwb7VpzHx~OjD+72J)qtS9c*cC^@iAHKyy%&FjcG74hdai{Nbom@HdN-?&UKlU5!n# zq@+p?gr`6^etE51ad|qosA=~IyU~Aqh(PX@t zr@Ge$?9kv@RK_aEEg7gw&y+Y3Hh~?{FAtb%vo(^h+obtCx+nm_OsE$_!h*;Wm&l`QqrR}{U*oo4nZhrh*R#s2Pk#Rnn4EJWKyC7Kk&B7e!(GCTsfy~Kfjvn>u z6!gK}&ukwB6&Q-pP6^V#=8)J`{q3%GT5Zu@sH-@Tm`#h#8=s*GRLsr?pa+`On=`M? z(ut1a!R38eBh%Mp;NeHfjT|!p2LlzN1^P@iv zNAD1pw_`2xKQ37|K+b0uN)D=w+Plbmb%!+_f9Dz?m%Gd2 z)x$v+zK1~>8@01FNt)fVq?p;y2Q}4@7u5l9q$WvsRLAyYPMTYGyydrtpk#5ApmDga zDsj*R&Qa|=#Jn>VY}-4N$GGpvh*X>rZXOz!Uh^3CE|qj^3Y4GCEo@Ym(OJ7IXL5FM z(@LRm$A|*Rz5(!r#ET~;c)T~undaJ(cS+%*6Le@GciD2c7;Tsac8zCZMHSGSYfZuT zJAR6AF>r|c4Z*hGBll;C&l{zq)WD2a2ML}TFCz+v-0EjDOw4@^oW5t=6Mg#B8FsjT z=w&Kx%O7v-m6ca+&A79cNt^?s>U)n$!{b*srAsbfD7x%$>}#AKGYUt8X08~|jVdJ5(+y`%!{pC`jw?0If2WjvFzUq~&Taiy4z@i{75$pQPXbV7fWEKOz zPij;qe5KJseH7uU7hc|9@H?6{W2Uq_v#|`(#A8hw5Mj+Q2?mWOsSmXjSWK?lVT_6W z6xCy((s0-N6T?_vcpda?MFz*dT|DgvHUyvpCd|vWbIl5?qc!yQIbGEa)Z5 zF~((k0sxL3E&nQz$SZsX=_ac>ByH7QmvUyfk{iuOnBQVYSs|Vec6ej2`2#UAuVX>e ztut{>l)~~)8S>&T0C=vd=#82>MEWbKwd|!?p#Q_0vA&c^4Y%7s%!>MblfC8mEN#8e zsvw!I;PyM%+aqbFYZ}m-66s5xEkG>#P;y>|V8uYlVu;?8u-k9}G~W^@J_11R zR!=a{*^SV-8V$w3_(i8$0F&)&m_GXk`Op!mw3d4X>9G`}JWb=})$@&NueRFM^Sl1~ zoNhW*E0Ak>?xZ2L{pI#r;+9@1rLC_@_U~^CiNd?{POuQ}1(Fp%5dF_h9~XrOBTtww z#THTjVt8P7%dtB$@HNvwkm_|BLc*9IP%`Ks4TlMWd7@E{@($C!Yl3%L>e6?x$%U5d zji#FHRq#sfy`JP~Q2HgsgYAIBP~MR6Cpr#oiW+!dZeU`^R?bn^x z9CfJA-&hg5F&FxS@4^AGr40x|!`>O|Cm<~%6igx(-SWIQ@KkO)7nriyZzwx>{uXIv zJE-}7g-Imd#}9}Z6HmJqlycZe;zE^zxa`mURyRm&Qxz>V`rVOsI2OdXQ}TXQe8O0d z5%P8cKu74<2rYGj2wA4pO4FV8c|7XcoC1AMdw$-t!X)vya73^5QM<6Tdse@cV0 z%W=9`F-=8eZS(d#Cbg|cHAZ()m4IE^g({0fQ&s>afb(a33hGS`4u~sOK-^@t96%(E zo*I8%3k}|rJQ&Ax8BYWtlk5xlGI5-%6J0B7Ty$sN)1X4UV(lI^nn#f73K5AQKa&yQtx)seq%M?d}Em0mLnAUKfb zZ6IlM@S!#o5eynQ#e{OGPU_$#v_=)l%_xd{L?u9+`e%ujP8ImI1oa*#d5YWAyZcAC zd1}9M$D3i9QNR(XVjZ`u7H;Ojar*wG23lzK_ko@dmt6#Ph2`am2;tgoC)vWJt6i;O=ds0&qY-ar7(Ja)MT zg)?u^qp(8h#>kw|brqX-AfdX#Wa{PASux9-q8n{{-bQ}(?;w=4rSxYWa?!P%ajcYs zKgdvlT*eY+P8U?U%0r+FGPlRJOy8=zC#c7Ke^z3bK&l z;HIQPnnBB>4A__*2s|PS!v&PDR@m?L`~K=OpIG3*7z*>jOP!q6H@gMU8nioB(%~69 zkYOAP+_x0ZRx1ISe#=mmM}q>{NR+9;<^6a*W6#9zQM}UH?A&}s7+9d z7m{cN66^flYz;Y}=u@b2xCAFzVH>wr=-OU6ILDR|$_ z5n4}+f58wY*UW?n^os>jqX9!_pW&5sxQ_kZrGeyjMd`24+beF#aR+V}(D6b zojpT%#@Mq3PW#2^CBLLmKuzU@h;o__?$e2Uj(XGT>h(7bC6*lwG9{=cUzz#-6KPR- zuQYBZ1f8HOg(*IMP5dm0S3UL_9dI93w&M-R;KY?!)#h7GXVUXx`bN^Y@Cg$%+z!(i zqxmFpC!>`wSg~{%c<0B}MA^;S7LG9*5s)K>{)s*F$@fqfDZRFcv`N2e`JK=;KnY^K z@u_hS>DUAy^<(e=0IK#(!p!FT$2~^1(WxU{q#xQ^OF}>vycLkp6Ac}6P0wq@OjyqD z6Fa$Pkh^ncGm!*4UlY+4N60u}iFjWLMp%U|H@RqL?O)raSDYwDBey!-ojMfO^?8hc zr@G61V(Zt0UtiLq$vVYNgFjp=8FHC0mgO3w_Ba_L48c^N73xxuGzq9dzl?8tQt%Ln z4U#QRDkReP#5yc&g}MQiS|HQ!-JuCZgJI0Ep7Hm9G6H&LNrD^>;MZ*-sbz_oPcfHG z3Ip8i9m(`7PGt(RdI~xy780YlS9SfRC@g+H4Q>GPGtV#;2QV4oqc!=B3KYeWxz5?o zzE^hNB~xu%-NAtt4&VRyA-A-w&9Lb&SD)zPw=>5}?kSqU_!j27C6m+s!*+sqF-fHx z;K1)f$3#@DtPOMH@;^9@_s`8}Da543o4);MxWNDC?5`bD9_C*qt6WZJ91PN~N52e% z5>EPOcO~smuVZmvk?f-n<6v{xb9)J4JkNtB{%ytN!MZL_Fe#75 za>9o$&%?$Xd~v?j8j?n1-}JlU%b}?Mj_8B6*Q^N1LN-XA_b5{tax-gXo7s1j`HlHkd5Q?B-6(?R<#>< zu=cC_3#WT#+AxJ@`U#C+pnKHDmQxvlzH8S2xQYOYc3O2PgkzZ{$%s{UW_QJnzg5F7 zB+Hb3B$%x2yp>rl(p59-bSNYZn}jAn`NGnTBFrJFB`i5ceQw1M6MP)H1I^AQ&eLT+ z8W9~On{U=;tmAXG!lo)oq%|Gb@M~5f(-v%^th3GMGKL6@j(qvL)2*032SwryDHC5` zAgWd5-Kn@{2cT7H!o$5QoPdia^zO8CyHyOTKpc`~PGdRTiklRUQs_)RzZMCSqzo27 z9Z&*D%+6ojVdz?k*E+{gVNFRP&_PY}y`sJo&aIZ@1mA(0Nl^i<-xJrrRefJamul(A zZCNK*M3(k6+%#|)>eMn3QM5W-ut`^*@YUtPfpr{nP7B_EqE~x&m=&9PFX>geysVSl zu{td9dJ>tSXi2QeKsZc|7uRd1${sM$X78`nk9mw<0??i(&;cn@Li!l97jgdpmy?Py z22gcw*N^5cYV39Xx+OkcqhZJZy2VR5>0TA+bVaQQg{{I`Wi~W`3xsxI!Mf!q$r4UZ z?4I!Q6U65;BBO&EXZh2&P9i_P%Iey_p_uFaq_Cm4yEfWo?FQ7Tk=SsVT#P<-4*(vo zZs<6Gam7eX@5}~dUwH7Opg2^PL&B>{-OKy-gR8(duBnzVsNZBC&5zy#)PVRL+XtZU zF}c=k!da1EJoR0&R~6lTiSGr9Z*GI*0dt>nc)8oHNhVT5M-fM3HVTd?F9;K?q~sDnOh1hV=#) zU*G9?Ms?VGAOWN88UsB9b!i;i-D0_x>{OEYW8LZMQxMkrxHa9o`8g)9MoDqSb2mD< zfK?#Fzi`NisJ`aV<6dVfSap4#KH(3JHB@$$Wl9plI~<`gZ>k)}QC5w!feWfi`RRmt z#u3K&h?D>w(#vUu)O z5Y<_`SNmruge*5QAxpJNT+vver?%PfN3*(QvRqK|hf#HULRS*4qyrzoxp2(>e4%mN zo4QF|wkttIp8q0Nb5-%+hJ@?j)@QlI^y!brwK}s*r%!Vf+U=?OnEV{vW3?CGXdAt) z7UEx}cA<#R~RoO>XFXaGWt z{P|eQ5<&(LKSvEwp{%Z$hyhwcFT_%b4p_|+mgLrDH6w&0e7xtmpgAUMqwv*j#7O`- zgaXJRI=eiv?bC|=$8RgBYG$p1>xxQQT!XV6^HI%v|LN8^-j4M02sbwB2SIp z?LVg({`%<^p!9xjRn;GUg5k}-CFw;{!bLhV2J{zw@gGL?zo#$$2UDU2%!vBGn~^WD zW7X(fst4p)r|9>c5dUypzjGyhON%Vw)v{s>lD0`+28KG0r41dHRYi>={BStHaB% zecj5h{m$_NK0l$v__8M-oa)OzEX_IZTFp3Pbg?~YNhPVgzdr>3%q_p^bfRK({@Ab= zv2QQ6k+h})+nl2LO8+=Bh#fxXm*tV_b+F}%0h`oVZm$JANaBC_l&*jIu(OIVM?$2X zmVcaSojEP%{*>J{BVh>11$ zDWx?K6F%f$t;+c_N&PQK+NYH-$u#-2u3L$I?wPpbcdNKxhpICz)mif{Pm}!1+g98# zy8gY>Sc^(dR#qs2i%MlFpMJd(;KRRpef!_>!(T)L8b-oJ1G9?V33m?Xi}~kJ&^GQ2 zO9&M)$lh^s(DcSClYX^1%-po}QK_}#r-!9+rVZw;qMu3)jvw-RF?A(JJ}UIh#9}yC z4`R4!=0C*f`t%m5sOgtXO6l*;fBl^7N%uef4P}3q-Z^DW7b#jlLWl4Aa11GT^L?{^ z^aoc$7vtPRYMlF`OF=!p7tpHKR^6+K+no5s-yNj-ZX#GEju&{jy< zOxImVqWF7lfM32r?v7TMu&W&yAfnIH;aII-AN}(yrhGc5OlHcz;{;SdFGZG}Igl=O zn>&4o(7|dxM4TCs!y`X`JOR~iK$-buwO)jiNQrlnd=&lcW9l{^AKrbng`|z8l^ng( z;0>)4*M~xEw<-nq9sNB0z;Lgj?A0|3^&Jqd92}LdV^>Ko6W9&RAc5O^nKwglXmB4 zo+{^tJ50lN#6BxpVb>8b+Gs*|RbiSNO^ls=^|P%@Dtf%WHcZN?!FJk{=N!EAgZ=C% zJ*QR-$M}-5em;(MeYb_*GP%B6ogF&d+bX&fsb;yD)5E=}^HudmFzd!nT<6 z+*f3vUaRGFjdm#OYBK*n)V+0FRPDMx{t74{(gG^dEdtWbNJ~pMqcqYv)PRHn5-QzD zH%K=~NOyO4%t#I}!2A~Oz0Y^Q@80L^bN>AOxjr9QGqcw7-1l`|_jTXT`DYl?cfyV< zQsB)U?cZ$zP4CeJ9{x95!pm!s66}3?w+PB`sW3^IdkC#m0USmb|-RX;Ciy495uy2yF26TNTYof)SG~ifa0m93| zq;T&Ili0F0FOoeOQNd%=nC`nq5LGFb13$Yd>o>5RO0~Z{&Rz2P71(&z{b4jum(|y2 zeBZ|I{P0><0(ss*5NObP=b@0m)2hC&6w}CC>+dIZ7_TuSe~2>XUK%+j2(c?X20n0*@GY^<3&`rK7rs%jkJyL}rt-*cR(Sbu8E7-7WC5 zVm&+hI7Iy`(&zEu-khwCL06mXryc1`z!c`MqQKzTH1Df)@57BPpP}9lU+c^V6L}SG zt}hE8hgheo^nUm(xRmC*n0+*VtkM@4vc|=A|5+u_W&<$AV1b8~S6_=CTq zZV-lbzx?}5rwq@&*f(G6T_RfQixye$9a#Kx-;-bM@2c%K_ia%u%j{QI0$j{nUd#s? z|J13oKyDQQu2)+0y>P=;aNvjd&%y(d*Pv+;{#AEQ?48ZBC2Tsgrf|1NLGI=+U3Q>x z4jiKS0jlQI#)b-1SKT_OJ|XZ_4)Sz$DLfBa<7FD41VO-`o&KoOMnsGX55d+eeR7KmCG zmF+1VyV;AgpWXzQBRC-!d&G?w!=jDCF#c= zZUoEdu(T+mZ0FTp&xb;r)sS71`47w6DEi$J@m{6Zh$h-N?sUq`ykU}NN<@0qu9ln=$yHs+m5CY*ClL` z?A32Qf5?r0$8+4#$z=-WwDgN+G43)oN)_=rZm8v4gS~nEsbHgt#SS2$<-?(^J3&-A zo?r(dmc2DK!asHIbw}LhrPiB(MP0?Hqhmyo7| z7r8=AuIu;TK!^uez&dU@)c~Q`mo{YxO)hQWLuIzrP>Y#tNh)mE(Za#3CPW9HXg=Vg za@}0&Rs4}wX)Vkg;jpjF5GcbNt7fU!h02Cj4bBkz)Dz6dzQ+YwO--Mh^!et!IZ!gx zX)S_!XZubHfM}s9wi${uz&EdlCty{H+6ZJ(IU%h8|9Y`8pl{3J>kjtJ(=YuB(nkIY z_woSq#8YapTyg;zU7L7*NbDGG2C{?>6hXgoowbhBZ%BN40nw!pG*14ZBBxy@4jnv%G0@P#U|ov$o@?D@|M@rb%7 zZfCMj87+XT=j<@yWQ3i4-@POk$sc}k)sc8N?#}#Xx)ER6-|JZ$v&Ckd65p()37m}6 z?pYx?wHwG7XR|35Kjj^$+}qz+Re0#vwKW8w0Bg2A?(;#wi+m=kud`|3aug!EenFP@ zwPw+iZf;?eu{J3no+D30U{m$B0_uEo^ky$FncD%kK{3ZTpa1-|}Z}asP%o zYxUgY)VPmIe9_X!7M>7UIJK-{{jQ|ClfPGHPO<6S*;cP-F(k77bm|C2K>QRiZb1Q{ z*Vkt@41^rS{n#O{iKdX>k`@I$)Vf%X5pELpZdo20OOQIQNu#9JB%Tkr9y`52 zs-%9^rWBbfEnnNYDWxA9W_xCz7>>c}6!wOqlerw0*w9Zh9+WL5eGs4HS51ZDt}>wAti`qsOsd|%b!cmo5JOjBGZW6~j? zqk%3?n%fFT4b-D5tJS6Jg(EjZZE2Z|q}B35m2+o*KRDJB=taLBiqHZu=S}u?pzugv z;APckOC+OI1f0pxGoHV65yrL9y6_!bZFZ>ka9X; zvNH;b!soa{tWG&U0upR(G9O4Zx|*O}+e!-5<-NJB{1;9Y$Bh)POaSLzTr95XSE-06 z#d(Otae!e{h2SOO&9KKT&|AY7psgFgm*6ZLnrZPf0+2OUdgO%#lQ*oext2(Ggnv7=194;^~Fk;Rrm80ovdvGoXH&56ENtE9XuU>9xe^%Q1XRa!T6fM2wOl<1&VR!B zbZs_)0Xf#QnJB38MqSs0Sid{M>)DE0KYAld(e+Kay3eIFUE<`rbP6>Z?JV(P(;wm6 zdYrAGzIXRJxZW)XFtC4z5AZLrB-xZFpjf$pk}lpS-9L*GcY!weMo)s+zsV5iYl^_^ z<+;f`Ih-B$#flrM*NPs1esF>SpxD6BmYc{{zSQc4e;uf=aj7{bagoY=(zZ%+n_}Ks z$pOBI3W&IZAqo9aNBQT*ukRh1cv* zH&8eRfGs}6y%F|3<4Hj7O^xo`>&pf3Hmyf}!9e(`?6ZJd_-Pysh~|TO8-|wXF;g$U z6jiJ&nE_F(XCEx4AmH`uG9`N=7E?pPVd*+J$u!dz?@Ta}+_fJt%qDOjD_Av6wBD2& zBsQ!wG1cE)fY2jl0%q!w141dcg$g?`2$-z@t@TQ~0$auMw<(#9g zHV!Dzff^g$Odj_s`LBLShjbrv6xHX4vhYC6;WNj2aOGTm`ZL37o4T;H7g{pmlq+$k zKtoWRcde^&gDz*!8(Au^T5#+Dr{d2%3`9FV{CSFyC-}Q%WsYwng92&a9K4YQI0j@7 zgnz@0ZGL@;cQQLWsBph@gUlCnst=+$LRHVHmef_}-u5<#pBBZ<(1#_`*IJ{}faA_S zXSgq&7rsenR+4_5g*{40@I6#s^WAnJUOHltW_;KSF0mbGgrRYzUpDMzk0x#Jltcv2inbhH^LRQCOu04ncf229owox`C1O zazq1W<0+z2v11m>IlR_d@!q_4B~u6l#+dpadC`fPyjws2VYjMD5p1%rR}S zt(#z>;oRT}Oc-AA0@V8&%NqilmlySPtzK-W$=wyB zslldvhZX)Ji-nCoGON!1{F9%X;f$&4TL3@fTO9@1Wzse3$~o!`wjkbg>EcQ#A;hLm z?|(L#l6IcRnkP#Gw zUt^wSu<2(o>aUcGlhlS|F4(T=;BHn7@sCbB(uKW0iYc10oE~-@ae7o{Lo<-fJFyQ@ zlg(NTfAqL>qq0Z7)O+UB_&0HNbvHU0JU--sZI`y!ccOCW7we0HHA02SBq7}zA6~LU zzK$wqknTGzaTY3#(E?W35FTQyI%+G1Ey)O!0ei5`AI-FH%3~JH7lf_Yp0Kdk1erq< zU{Hwl?+hacxJ7kUFZqG_=sM>K+mx@DgB}yZf>|G#`iG+!MPYs51C-(gdPlr^v=e6f zrczr5Q#?nl@7~vi|6KBIkx+l^q-Vjyqb%1no&I21CPP2KO^M3Mal`q!c3j`}kS{ZX^j z*YDqu)l!LkPc!npF7{3PSu;TYQY7>sDo5yy_K#NRxSwv1utQ(0{dvODT!-@!%!SsW%QW|?pp*LP@_kCCbe)7z zeEFBy%y$^RVMM)qID$J< z>cUnFmAlVR>a8+j^HA8OqdGcwVT(TweRd>21PEim+9=T4T2PJhD)gSSP74Tnhd9k= z5Ff#>)pk$J4D-!P52AmSNsT<62iaA|rz*t-*)3f4)i$jO5z>7cp)z{#vIt}Cmn1dC zp(7iRO6YktSz?m+Zs_hT%^b`@EJy15c;2X){;&$h>NZ?vm!J=xz^4-Yv3i)xVw*@d z;0nadw$)MG&Eu2&6&Wdq`sgI<)$E#LtXWJqA$F?@qXpHq=rXQ$_XgN$5$Teh z-)lvK99_k_(Fz~3s3&^)zZgpeM}4RAC1O%m#NFjhrx30gK|xRNhGrEK8oX}b8sN&O zmyZrMS4@+YixT;HfeeUvD%det|Crh1D4KC8SfkpTlbFm`$ZgxMW0t^~E=^8n9yiEz z-SM>!Gmb=yv7g!7%P(+>3>%CWKcQjdpf;@dlxUp6WBP)2A21pO>r-FF`vD(lGQayP zkWJpZm^Xv)%*r<_Pd$&rA$c!v{CPe|hpn#K(Ej`-+xrCJ5}=JQ;9?0f8%PZJ0UU+* zWTL#|v#lN^3m?SXinJ%+IxEyJ)oa^PN`uO0c0J(LCgP+R3?fD>LcN%Agc}*nr+mME zq$X_Z9ov^mVNh^W5306DC3v1i5cNn#i$Zsw(mnm~{pi<tx~_k8rLGFG74 zCE&4)ttGoHHeQ5363%k8UCZ<42i)PlpC9oX3v{(yIq}h1R)fj;mn#)H6O7x=6BxsB z_QP)aIC8N!J_uo}Iv&((KEU^dbtNx%QGqw8I-`uJO)(+Swoa+cQG5e9IW`M(t4Qfj zY7NzfMMq7o5f^zl@maJPlN$M{*&lHebxsLc>G4j5SpG2!;D@UBMuD06s^6Fp?djl6 zSgS3Ps22GSKXtgL%6sB)crkz(RFgJg9ea$C*+A=qdpeIyh7INSIBirkc!EH*V$INrA2n!8ji{a6^EH*+(z;ByanM092 zQ2lN!-mv&{Lha4CoE#aG@lkh=I@5sH9b)3-N)oUc<4zol_V_xkZ@mlbXH?|k##R@b zkkbjZ=uhKq9s_$-3(iAv#!>+`eb%=iv5xWT7ue>Pl@~d-%erpczwUU3Fb%%k$D52P zjIBy-b@Em2{h3zl7fo~Y9$R|L!#_~bvz5Er(_x%mV7b++O8Ma9r1LT(NSMpaGXaGk z^W#^X#hum84y-Pd`woR&rjI;odMZp0BQIP|w%_`sH7OZ3oN&s$9FN#iJhKrT@WNOk z4;<&|l3He;s&_c__oy))h#Y9_Vn0H7;=YTApsiw=V(3-MP}Bo@SADV^4^A_m{7|-Ufb?Mfz4+ z?N;-7*&c&)Obi~#mST08Cn|pqoXmIN7-&1%apM}x?DRJJkCV%UYx`D4m14^S)Y&iI zVbA0E&Zm^O7|jXDwZ>rGLFXjVE9JIRab$7uZCHKdc`Asv89dt28 zIyr~;&u?AeA)vI=6!YAhH}Q*wM4Az-oiV&=Q6L0YNMDQKir5u)7M&RTT^K=)Hc~)3 zd11$&Hx7-ojzjNzZ*Xye7KgQh8s{l6TOKn!B(c6%jsa7Z<36 zJPK7;e?4Y?suS$pKCgakUYu_l!evSpFX_(jHAySfp3&8*w5x-FsXU>5QW>hE&A#+z z7u<436wt$(xpx>~DbTc!%fy1hxo#hQQ6 zJVZ5qsMs2;pAV{WpI%Bg;e@eO)8E)jbV)o!y=Hd475_; zRNEaNs$fMt{G?Oey&sTyR4C$+p<+qK-Ge6VT<&cBNVrx)L#M__lUSw0g&?<0la!V8 z2b)_@MwQG1yhG!?Y9&PjLQaY2U{Qg0VL7^Ed>RF&R;1go%rgqTME7$#e~|`isL#36 zn$S|C8M=_iQD6vCHJ?i(BIO5-+G70yQ(qG)X09^1eaG~9SPYi=fR5$Sc$0;!TgFl+ z-t^$y?0QH=*sn!)Sdb2v_n=x7liXMu($CVT*A&jERAaZ0exF2v8Yw2Z(q^yr*$#Sf zal}7SV3<&1*kV3Nc!#xldT#Y7_*AB2Hb#2hvJZ=ZZ z*r!Vr`l@g-of>?9=ut9UVL3to>!uR3F1A!CRn&|92@zj!aIN)3NMiDP>w(m|FKRi8 z(oIMg&QrNmissZg0siWA>0>^P40o$NL+YI>yNR!#e02O@%;Y(a)tlFdm{9AK^lU)) zCfO>$i^NA_t;)W;Sy{;T&r9kpH>#^TSt}~~;@ZI>{cILm>o`;WA|yvM^vvaK{_$?>5FU0E(;>GXQ(g}S%YGcvS&jXs z12^kIjUDpqo}+e@p<=2|wM&Xe=~R*7NNw`k#fPmw7>TCm&iX{etvbFRnzsO00zr{f zTw{TKGxR#)=UHDYN4j+AR)PYxFMx~Vw=~Az-LE;HIe7%vw}_ad=q)p0RB><5La)6U zT8_~r2rL`t_Z8?gqmVE%_JY5nlZ`OoaUW*pPS1FE>cr!Aq9Gp#V`%$CqI39%_+pQn zUqcimS8%2)tr^Fl=0o*(9|{)vRa{rMF=l5ullhSkP5-?n>8 zWm#6%LRhShDmV2SS`x%T#h;&GaS5M%{u}1}H-YUvk^3twccNNQ6d!+#U46%V>yocw zw#Rkm(-cv`S~|7rw$>BR9V5mYFuzZ(_5M-HRb7!xrB5h@J6RwJ-?i)APmrv6I1kTZ z$(37}-HmICTF!tS2G?{x8YH0eHmU)9?wHKBuh2jSTa@$89b!U)`Ma=b?UJAJ@?S4W z##%1sb{0`mbne6%ytKh76xRdJqOJ9?1b z{ZJub$>^qgXE4Ec@q7%6yf~#A-(kAsvBBrwSZrhdgUs=AO^t;^Mu(b|chHmAkHPn( zz3*c#zLo5HEPKLtkf^t{CYdseJAGIs6#9`)M?}O=uWJA0&rh)@%x;Mw`vgbrn$UNk zt8W5#r)pU{AacnqPc4Eh9iO^_HgBLuSu>p}vkMF$Y~dh|_?9;5+ItZ%_KCJHh-=DIFuPO@4Rxifahwjqf1kgHwtAzY>6jP{YR=M&OCJH9$ zA+6s?{6H^l;^7ll<3TB65ZSm|z^sWI{+NGWpj{sug%m$RPq%R>{=^<+ zJ@xKo4C_L13p*z2dbL=Q2ID6;O`aM{+!*y0^lWBaWMs9LFiwFoWnKj+uviVh&+Cz6 z54l0!oodgz!zA7@;@oaA+9tiio*P`NI%_=nymZ)s^gC?2F6M4Sioxo~v$qz<-gm9G zzuz(6z<7GX*zY3vwZ*-V4mR{i5N>pTK4si-1#Lv2V~zQFfBN8b zcVwwH#fwNon1$@^0lgiy*>nlid5w=_hEmWQt<i+=i%7 zf)?KvnrMC%A8TGrIl5~7+#w5nLb2JDLU|cCa9W?bp&z3`|K6s#Er4m|=H9xYI2x~+?K!eo5;}=sReH8VJo(7<{ zYgcS_GD-Pi<++x+5my%N-!Q^FBEvOXi z|M3)WeskkF!1_X}eK{vT<^GQVs$J@()F=!#A2aLw_5r2a->ITq4?AFqILEy_-j&|d za&B69@6M}Z$a(C^2uX-OcqZyD{O(sJ{=pNye)x$L%< zOU(NS6=~6pzn0o3nt3F4{0R?YUd~_7mlATUDr_wGO<^i$W%NM zkTh@JB8^>Z&SI5#YZb~Hu%|ZhOtsC!{y?0uj2hRzO)=NC*dQh11pXUd5At(lT&q*hvVd4fU%kgi7QXn31v??b3rV{Mm1)F9eegq><>*DIDP z!t7#RUJ*WGh_<+-7m4U4d z-r9EjC#~ss-uEWP$^sjzY5FyM2`*$qZL1s_kglaVT}-uiP^~^QE8U^r+PgkRXu4$x z?eRuX{uZhJEns+!hLT463_5(spp+1yEiSUEiIGCRLL7Z`VdHXucvXMn)gO_HlKzM| zN=6pi8ijh8GWW#zo1$_+k+zCy#9G|0FUMY~Z5bPh{@ zQl)dlQAy-sX%IxMt)4wc(*R4z1yANG;qohI6ChGSzOjR;!jdi5ZmH6*vwK2*ETr+w z|I9tz{rLLzhk{DaQ?5i2*Tz0*sO0WQW>CKC93y?(AxuucV=NhY8+Lj;&_ZJTsf@e6fj_O)z?yh2+Y|a{Tk=a z!Uvp|(S!<%?eoLTlS5fE51p;A0uO6Txm~ZP59uN);tr~;r}C_8+$NiI)0AZf6037W z>cG6G*qF=AOE0qS88*6nm70hA8WwxE)T+6h$^pK!J7@8thcnkK@g_Tr$8KLE2n_`T$+8wdxP8$bcr zf1qFaTRhqISEMXs3cx0fw;ZYT5EX)6KE!4sESp>V^rPu*=9^DpVLVat)DOsaNf@-*&Rlw)f@ZsdS?7dGcN=cPDpXFoHe=2N_F*)ZTAJ)#LfxBf5s*0H% z9XQna91x(sp*HZesIX?QMj#}E3Bc4&grv%=#z&GieXlWK#EcOcu6x?fbD^G7rMeA~ zj7pSr(iYXUJ}rP?meBt`bPXDe7}yzO^56}bVrFoT#kN$&fVmwuyokNr$Cq7aDA^?# zJvw{p>Ir{^erMGeYd(|lk&eGiL z_bR?z(^uJ0 zNUSR2rfXZKy^saK#Ha&VU|T+4*h0+#`&8u2lL>3}(|1*J7>?l*iK+>s;v*fu9MlXe z-((XShvbo3O@FA0%z@&_!t`7-0&C8$Q^izLxPnquQa;ZA^phiEO22!o{U|D>d?PeL z{lj(9jJlOohlEN`-TpEC;7QO<|IYC?P!bB=%waQbz%*P=eKt|?{3>Or?XZOf5CTbS zQRe604B}*qwuC&oh*my*n>SS48gDg}8g<>(;yhi-ruOz0Z`dId3LMS$3K3iQp4b^q zWfp>*XTqgsA++rUw6b28jU(KBM&Br=V(<0HLa7j(i;U^`YxvheZbQ z8V=fimpO_9>q63le4R42E1Lu*9MVBYL7*B?m*7HYtEy%E<36#&;2DVsHurTt-NBq& zHm$T+*yh89Onqh&_Nz&h;&KF%l=FP}Eqb0w5KJVUIr2#$hW@MdnU%dd+Z2LO{3xcf zV9|llkyNW!PCXeD=9>2%A#oNV4M~El5*^lHhSoG8mYw=3Vc*&NN6R~pKV8FiLalG+ zeeqO1-ISDUY(kYituRVN?#C%#XV0j!WVhO>XC%aO@ZZ@Y* z3~rY`?x*0#XmSk%I8*)(RCoon62!8Q;DPV{-tJn_B(`g!@A z22%S>Ey*9EN+}{iJ`KR)!&O;T^o>8;2*J8=muQQi0CC3+GTKx5yiT62gqex~(la2z zeWFe-(QA5c*y6KLUo-9F0A!6JL+Z;>g&nJ_X<#2g3Mik_kKAhf1pr6=uvoLIxV+eG zNcQZ=XvC6ML2-L~vI-x1JrH*bcjQw=?uG*t#?HBJ(*AYIhwvJu=8lcWtXSWgKh@%6 z=Ki{mQRvQqhEYWM!fR7P-Pfx^;abbB5Rt2xdt7key|XUVz(NscOA}j7eaUBGrz@@c z%NSBTHIgBKyb0nb$XN5>@hp`p^j@tZms{vR! zD#-#eYH=|;UzbAsXUV20~y*L`G;t5yVI&0FM2xtX{fuDu9f4_aF;bfviwkzE2ubk%&8(0Q!Ua~Qz2vr zqX%Yh7I!XOISbC(jVkrG)~vk2-`jgCTJqPrAUB%dNufgTGY2xFdd=G+SL_yjU&f2WC=2MNy;5BxKE-?Wa&|ycE%Vxioc|)3! ze4HOpXn7p8s<9kl{gVFXWUOa zyqL^;%uDzq4axap?blHE3i0+vT*VWn3n=pJ_0dBXEcOnpH>UQX8@=~b+wTo1lFQDC z3$7)6K-M{~-P0^oT_jTSBC4fR%5|k@C^hG_-b5R0&&67ZLuYcziY=Tbo*~ zJjb-V)xWbMa!B*RgYjDlP~u5Y%k^1wuwKYvNfGeqH;@Qy$lKz#Gsrb;fc%obw=;9+ z=B|1Ug-|umkd)L}uCjbREmkb@Ft>R5slC}I3p9ZMquI-&TzWK1#kpb7rq^^dy38C6 zee|Tp#!=ZI@SUc$szbPtIZ+2-v2aIzcH|FQ6Y&2pz}PAMxE^QzDHekaK@n`z37##|574rkq4&ykL^yC*GXwP< zR+G6QLekImO@Fes+cnYPCOz?^a+BzevEsp5W7cllp08f<(xs>lv%_kk z8k<7Y^AVNkM{#u7M`cwDZ>QCz?9Hw{S8DRSYF-bHsf>p%3=l4-;^gWOIvP_w-QgU$ zBe4M#SKg2{9V-*=j;CWb#Q6@fZ;JMdT@ljEzxQj}L9#9BAn9U@HjasPw$}c5YK~b} z!Lf@!D)UE0HX%-(Z9mV_A?D$|pm45L%a9;XLF+S9(gYd7s?b{e2-uLwaS`FwssYv? zH0!YOx^Ra1P9NXihlOlB7G196Y}oNSqh;!>Wc>QM2s*2d2~y*O^nfm%)i+sQmnU+h z>uH=rbhnSk-}+m)_6zp$^N2s<94Z~L78!znG&xW4nL7pVL-b?`s9cNGyI}KYl>9># z75xXt9DIo+cle-ZEa8;g4f8eEHn05EMwZwNQsyF~`LGwE&qT`s_+A{U05lRWpO)H8JU}2w@U`L|BAZ z2Hg@w7InsWIdmfJQP_(d1H zIz0yk*BV9oJ)$D+{grwQ=v`{;h=vEkZrjT6=Hq7G1ZCJ=R?OwPkS$EK=`|D`!KpA| zt%*N5V|`tz5Z%sP4K@+|w--L(w`gQ-_erf(Ov?DQ)E^5Pqlu;og_B&j{aotz5(KOw zfV~A6=JHVbXdR@>at|fj_Dmq9zOqVdKwKhqcMi3C@pR71{^4lYlD4?a_|kXP@!bct zvbqDnU;aRmh%`l|Gzx!>EqzZS#+5ZBB_1QM8-?YalGzpWcMs1$(x-#WZ{nx>e4=tm-@u`cHUkNC{D$@3ES+wCm-i4S;K3`R@M5faX@|7S zx~&}#QXIH(tCynLfl9(+v3*0G;SB|y3=@Lb1ntgK9&e`&GmEuM_BJnjIq|D_F{lin zPS{tv?H@cv`uOg?0_G$2I}gbTSAo&fM5{0bQ0-NMeBx3`FycQ%vUk?&4t$KL4~|D< zi5)1028Qy`WgPZXbAmHD3IA|B-h{>|r;0o-7OhPSf|nIX-BgmO09iw&oTIO=ba;a` zPN&G*$W8Qxe&e3gu{=Rssll1hN_U#mAybKP;B;)dm*+X&nRclmgQD0WQ>FdXhfj^# z(I0k+b-q)sY-z=@O+$Bv{LOhS_=vUXgKLBVq@u z+g_1YN<7qPH_@RWNA%u)sE{0EutuR#1WyQNWW_yuD`km(+O@B%W)hHI{qTX_hM)zuYfSGw!byJ#qe&jqwHjUS3XY#A*U| z6_>(4Kazh!+y8I|{=o5jDYUv)@nZoe$Gt{a+#h4SQcmI7`7HSfFsW+{=;>lPy3*CF zbf*NGJ~rJUE%>vzqL-Fv&vd@O0Cshd59+!%AhrpanzldVtQzEIxC4lLRsf?#`aqkO z+r_^DMzQg4KE&KCOvV?+v*jCeUXh1h6JodO@nde&oO?8_2>PW&%QqNTEM&f`_hOTf z&RW1>N!v#;3NDu?`ABE+rcZxe!Sd9Uzt`}KydQ?Iwzd9eUUBTPu#P5S>5T>%Zw=*_O(Uoq5QpLa0g_mX5r060C&>VJ~F|Mp6MQ4+Fp=$j$dccUl(rgzrMN6F-@IqnrS=tnH&KobxYxsOF1Dx& z)Ju0o>wZVbvwcC!W3r^})$Gta8xwPKrKSTig3cT1L!hNsadnDl*t`W(Ib3GXUPpGw zZJ+FnjTcE@zl=Qe7!-y-OxIvh8v)`XwDXj71;8b*5q9a({6mG&fXH*1oDlI?IBb>H z)_%(SHR4tsc*y5~A)N}6eH(zh>^I-}-0LhDms_?bGYn`(&`!vcbZ7(n$zlW5Yk;cl zET^L@tAuJFF8b_=%Evq-Db{)X7FGxdg7-^QgJdF}NC3?IVJS@0UgqDtqMI6wTa$7& zh+TyxGWXjCYiy#AAIdi3hFsfsx(@R;cp*?S3zH2R|0xD+|MpsH{Oui3h}k?h8(7s{ zZg!q>+g;2Ii17qu6U*K#6|4`8E4OJFFnXz??lhTBLFZyw4E>&b;K}7^xk~;R{%9NF z%E?^yX7Rq~`d8WQ(w3VnK>c{V#^`#b+|`4<(C~3|ue3!vQ#ZKDYRuDFK1Y#5y$oMr zn19KKgtg2O)nB5VOaiV9d4vfzL5jDITJ}n>6CSkbRF%c(N zL8f}N&XTWKm!TC&kh9et8%4^I0DpDG@mv*AV%Q*a+eZS(c}l6H)$Bwnp!bHkc%uCa zn#D$wTkByn++=Eiv|_n8NtF+=oDxVUnuv%U>xp8@lP>%sAb0#$(G>9b$w$&-$FPC> zO;KfCl)2J4S|jilFdwl6WvADu$iLZ-KLTXG$s(FH@|MYSKnhNf8*h5vcWOy?3-t^Q zF63PptLO+I>uWAkc9F!67=ylBRV_er|-{8df{hW?VGmv8&2dZr9E;GOHSUF z3lwwde$;y=kH?*W7WK^g!^H|=G!L*zs-fIM-sj&)7Sv}aTiq{#%|VqN%Qib!=WGEW zC5JRf0#L-ftkuv> z9e@n*Ri!72{!=1;nfx2kaD4-^-`lx1-bWeQ0+Do$=f*tNIQ zms`4|>J|y|IwLz`a0puS?hH%V*&d-w=FrAubturwzK1H5OXByHSz`?2al8#$aW+gq z__{5|4JM090%CeYAE$$ikU*E{HB+klD1#^juoT_ZC?=BZ+_ikOUKj)zlK1*b50V0C&Y~2 z{RPfWzksL~--FL@e`BJQ>Ohu0SGrUKhRoJk2gxm|i$(Efr8YUTy>CAQF5yf*-HXS90 zG5i7zHMa(vluDW&5bW*)m1Gw)TI^MUqmShhNQE>pI+fRgl*QL_-ZQJsO`ir_i|QJ#S%TVzDpfjN$j~X0mVNINpNsqQ zaUjnof%>!ES`^bjMx~XJ%VUFzlqXD7j@LkU)JT*HBiD{$)1@%<<|kv+KccavA(Qr? z$T-lNCb4lFko#`{q2V!EVP6~&Df{Rqll(w?^fwOs=Y;Pj>dR&{s}=9Am9S5>4+5R4 zs>ld7xlwoc0)Cz((ysD@VIi^tnCSW7IeUDI|C+2?z?U}_bB4>!maWg^qnUS&)*I7m zZeBtq8hWNI^&jM&!ubQsE&9x2SdYA~XL);j5c@EqgV1(X-HMU#>1!E=OvW=3i9L8r zSO>FiIl%ErOU8F;jt5ip^JPm1Cq10(>0~~RkR|c!BKzWsUjd_LHlXBI9o?;fy*DtG zXGi_{eIxUqarh>kc)8(8lRetjbANY%?lpK@j&OBgWgJ-baoPw&cI8S*m`x{A{@#%G z4d`A=vQ&Cflkpk!OaTmSO8EHTz&=F?5%n{Nt=On3j8TQc{-7(10IRrAgVjXoy#xwz1-AqK@xxj6PFIdo$4f3zMZ7`*qoKBzgav_|0F zRt$ZQy^@$3lUIahLIBHCqXMOO)g7r=x2ppPSoI!hf36ZTLvC#_Wv=q=njZH>M=CSM zd3yTW=*^2t(o$UVsbPJ4(sX*c$X}-E#%J3P%S{Cio&ZZZUn5 z?ieM+UY5jft;C=ZogPz2Yq?vA&E7TN@~tJu?SNp6!_VYJ+4)syc+S$WXo(NgCzW~h zonQcXR-^r!IR7V{e_r}KI_ro!pdk>u>zM{{i!9k`U4dbTpp^FzU&pZKE|9Q>pQZ2H<44ep_mMq6L_pUz3V*>@P(Y#Uu0WsZG&!V<6QF%+^Y9?=%=L;~nk<9&!&*Q($Zo5LlSeK#WOKBp>OQisMFQYXO;%+`d`qwo42) zRkMv`1C#%M%)Mn;RsFX9FAai(NOyyDcY`#DDBVa&gLHQ{N`rKFcM8%iwP4ZRAdBWV z@p+!T&$G`y|9#H8|4XmS3)Y&<`JH3jz=Mv7O8W?eqQ)0(+Dy;9{N~qm)yg&B0HM|`8EB&29yD2XP=#*Wr_g=w zKb7J%n(W;Pqu_~rg?&cUcG#ons#`(Wi=$boC4c<|!KdlG=MOnMxx0#v>q*dU<0-`s zurr&|u>&Z#eirHQK@3+xhO1aW;9CQs-UZ^LW@PUD6xp5Wp0;>Vc zW~omR+P2%vlaGFMlvaqh&IcgH=$desr>q&ftMvN9M8G0-m-nN{#fu3qz`|^zBtIK# zbXlFk=>U}{dyQi7au}$!UqnaXRf=^kx!3BAJ@5h|PO(}8MwxC+8nsdrzeYEhE5H7_?D|TE{K@Go?Z|(fa(2%gkBv_xj3;{;~<~%4Xlgx`dU8c9hb3_rXxbSRE zhQE*f{Abx>?PDjg5XAL-xc5{>rB4$Ri-pd(@4_{I>5U*anw^H}w~Yanl;)5tAe+gW zLxbt^6R-=e9iW^5z9+IuG??0rPk`@+LC;g-Xw?fE#YDI9gJuZjTdO{ zL-ZMvMep*mvFIsUB}46mBlMe{kvC7n)FP{3BW(L@!53|az_bzXNYArw-t-JG$Y*yv zK7_a5b&M&1t+$4mr(PdfNOe>NT`7@{p9Nzmk7~Q_oR4v9Zpl7O7Av=LeKDA#j!;M! z@=QsoD?GM&8MW2{%w%N0kYh2LVrB-@^=`TTVp%e^JP$iZoOTT@?9O#vj<;9lBQvT>Eam_15KiKU0)LMAb;w+L60y%f!8Pq2~MW6saig|Igl11cgDEx zUL`bK(rU=&+xkARtG%J|-2DB}M#bAK?m|`3+) zKBvDr>|@J3jrX{#YMZ82b$G_6x@E&oY0x6y*-SeB2dAS2>$m{-$q<9}5wDg`&oc?9 zcvFNeIOOejSFIY&Z+3){U z5Yr%M_}K3$?-z3u3B@EYCXy-=!)Eid_QjXi8Vi~OB80;eihJCp297svdbzaWkd~wY ziW!UdEar$bh&8M-&`ifKw}P3u$qxZVc^v5PhL4_{i1r7?@T*rpt%PN!&9rQwot0)| z1wuOSiwZLGT^%Uobzgs=vvT|-Vw=h5_{quc7|W=@?bb;rBEI>Sw6Vhqk(<+T!G*mE zI&<~4utlF3h5hkz4Mhb$I_D|^B&^cE1F~}e@*`GgB#16xh8{hw9?HBQ7i2T>x_1p+ zY>aR;O()~7egV$Y;*U2$IQroVvQq}>Gtlw z*IHGe0sGiTD03usTtfnR&0 zMQii!fl@w3gvF7h?XmlJ&T*PdmQ^DGc-H>SIQ8rE%R_#X%`q~A92Y||rec+bK^RWf zE~=gzDn%q%&4)AV`sm}UcvigdPTpBpklAfiSw8bZCP!MIBZ+^P>+*vC6vwet>zqKv zk<`Nxgj3)PUZ02>!|n7(&8u-k|MpcCWCKlXr{}zis64Uy+pKFL#*;^;3#xbUW`e%QEyCl5Oqlno9;I;`U zd27C-$?=e=6{>f8c!v@#mZ%n)ORUwHD&R>rYc53z2+PG6QL>XYs>4yCp_Gp!#}bxFi#Ato?Q`*DG~!#rGs-R{S3gG0t8!vR#s4OJQ{7n zSuVX0L0TL(;#6UY`Aw_r=0Lxn=#FVu+XgdDf~p9&b#DcTh#Rh?mp0n1GwZOFHDz~V@6l)hX51s9i+oBa;#A0m9Y1^#|JMO`@ zOmVQZq(4361&ZmP?drQE@Ip42_1b^e{N{=o9E+tX&kC-PalF0^5tydI3CMKEcj_!0 z#pRv5R$4VXMtS|M=>6*-D`>uNkD>=MJ4holSzf2aV0IjUZo3W2psvVUPlfAZCGf=M?ewU6s-G0402J* z8o0_yLRmFjUg!}V3CRrY-*VmD&La-nSsp~`Q%Yt#ecfjiEc}-Ir^w<@ab;Ynk7v3_ zp!6s#_o1^M2HWVvHg}`7c9OB|pkJvIePchT(1bmg0RNmrT#fzqtFAoF3Fhk0{J-n?gwn2cqZaJyc?;U`5B;J5#cz962b}mzOAM2D zTK(ZGncIAF8YPoR{yAgRj+*aSL72o;5-7h&IjyD<2eKx(MKZrkgV^Wbk=s-0Rb9R-hf;)qVxRfi5`m`_cMNFiA&F zWtXQ0O8eK!=~Nz!NPN~-n}(12!bTX&jq0bOgCj=Uvd0yhxXGNbzHIVp}?^ zgMBR$6^}LA?_$9XJ3Fcny6G*7>0BNh&KXqLaJlma@{d73F{eAn}> z0wDX-C|}#v<|ho=TIw0xByd4m=@?=4Uhrvfiny>7Y~dj_9B|Y9PBu6JzQ0Kl0mSrf zGt{HHPe0dZ@coVlp5q+3j{m#47`iu72+vZy&*8jjW)9xeR_yJDl{5a%c`7R-Xm8(7 z#BlcLki+Zlhff;*(o}RbcpdXeb6qd#Y-5JNY^^_ZCf(#^0_s{hxY4@o`EHR{4?3mi z4GPHHIbJbSXmbrZx_xHb#S!j8?eWHflGAG0CmI|^jUPexLVHTZJ429JCSJ&R7)M`s zoxbyAy*m&*EJ)^UOD3hKMcPHuMO*%BQVwspIHX~Iu0OGp=-{#GI(nc)&_*8)iU{`T zN6W=TGuWRw-}a|eWkqG^aX0L3tP-p{n4TJQRoiA3LWRzb)OXTx>MbXsxC1YMdHD?L3{WOToKh)nz)arZ z=$QSm`Mj)$H()u$-cxY%y3}s=7+A+`OeZNPQ2_`-v-i4bU&%)OCtjj_6JV_?K)#GV zYq>a>*K?`<1mXt*Csdxdj@vWS&Pi^h$6^D>3k(bMc-U$!1ddI}AY~i_568PJ<^A;6 zyZRR18FP@K?Ia#q!h*%5y|>z++ohYw=plB|g|xdzrS=}Ur78B|#THBmUe(7k*9W-S z;+-?^DH%+YNo3~S_0xm$3w8nw@(^E~K}7mQIgw=jG+@3j^`O%vu=mKvdfL_0#ku#c zu9#Ctb?qyM}s+rIdI`7-~u4(_-P9S>nx~GD7f1rsekvm~Da$CM4EQ6W4 z7dzSw>8YbJy+S=~o^U_*6aj=YWqG->Y)m>)zkY2dxPGHMQV90m>C19atl~+avor^2 zPD~RLcTA{9S6Ko;pdE7?2q-bxZ4R}2x|^@%PHdHM`?iE+VjvMAUuAiv@E4Q%DJHXh zp;Eu3j*twd0JV(OyloVfu59{R;EQN8%T3dGkKyayKU!w?_|V2AwV8wny5C$;SYfiB zp`~d2;19z`F7!~yfSg5AW#aGN%>9w&&u)A+G*X)zK2<=jt(xD%rKNpHVWHi{A< zr6a1V#UNDT1*t>eK^TwuGpm)j1cV*ojvi{EYz}cDYnh#%ZFb$vTJ1^tEQ?zsnkoY? zja_yjc+Jx-$aw#j=k$Q?(jl$lDNL#z}r>pdBT)QhMpJ?4Cl_4BIVTJ+V`Yum&xqJqi2pZDhcpSP>d?NRuP z*qtSuGMS*;nj9~1CRDe@Efqhoz-r*Lf;DB{-HenX-gPl?*vG+-{`#4T^Xh%zYx@q~ zn^zc>#k+dzt=~;3au=@7<+km# z9ZaWZo+pw$^sKPuF4wE5l4z)}_A6laO>*>0MEeI3?li1fMnN8N^-UD9{FCsoDPNj& z3jpEHljOu|NIVyTX>lHL8=ux`?lGq4l16pASkZ%y0S;!`UnICx#2Nhw?f3Y&Q)W}; zMd7v>@wEC9aT&K=RWE6~^W<>M(FwW_78@3uu>;(7_#s0fr51m;U%Pwc0iQMVrD5-<0p=1LeWk1?e;@s+)8X={WX$n}mweyd z)q2U@1or8{Kn7#tcZ_xQ*$SPUv@~ZtaIBo6N1oyM5*MmN%9s#X+NeL0rCYkR55<8P zg8m!%hQ+D4l^Z49=0E~IH`d=M5p3K>25)M%@9{*Kyrltv8e8PcL5)sFG;)#3O4r}w zRCVmUof?H3(gi96HfKFa_C|fY#9g};->3wcIV5#*r1ign9JMWCpG3%fWeC(LYjB;a zS9Q%<(oZ|rDu29OsRi8F04yW_`6hj`6QGZLN@1~k#cNLAJ~+HsQ{_;QaiQre_R}3> zJs%*utzh}$Xf>?XT9#qnU9FT@WgRvEe!2|TEnfN$0_*T-yB0`Zb|O~*a3Ia&3*ab= zt?w}Aaa@39_j<0(1O4CeZU#Rtu5nVIW*kFzFSc(D19+C9|G>KS*q-4~lmh1Cw`D26 z$~;KVTr<>XGt_)LhB2R`A3#I=5iX~Isq(Mmx(l*a}nS)_7+E;M^pD86K@Ab5-PFUUgZgpdVj(O#8LR!Ip1%>10Dy{4cS1 z5rN-s3{ux`z+3;7#8K1om~?dK8h0lkU5h$-+cA*Z@iWm0BQ7n~+M%HK*natHT+DLy zq0~&~!M8aBp|f4`75z&8@Ordxmz^9wcfg;SKG&a@zen;|Xe3)>T5q1DWie37gqKH) z1j>pW5F3*Zj-?4UwW~+><&O&vKiBiVH*RX^{VZbx?Qm^3o1;eK)H?Txiq&&Zd%c+= z3KaOfO;oT2!oCK{AKBMB{kTF8u!PDqYYPlPp%_9oTtE!5zZqMo^AtXP=@Rv=H;>E9 zBqJY(pApw4L0*6ot4c%eWyTz>KIXU<^&xtp;ZRmG%ma7CSdUurt^)RmoFRiiC?1>X zxw8S6;T+Fk92LPl4GIbo+fNkP!=;v?&+wh6uyVZv4?V$XF9TMpNsz)~+6<@%4QEPjqoeLcfI&HNeB`J_W4(^_3&y)W86+R(EBty zkJjFA*S?S@KET8omb{7vB(=Q+-oEs1PiSSp5n|SLhxG=1g~|4KK8FPB^%XqY=xq`;1QKmxGltiw3uJy3$HAI zuf!wxOBnM{F{T{uh|2%=Z9AkG&va3Q9s?xN;MwH!GZ=!zfh32r!DNd=_Of>`IGRxZ z^lB=BYb1P&FXs)Kf!!|rx$DieEk$d$7b2~ z0McZZ7xqEDmOV1~JQFJ3y1-anWnVR2Uhh=|XSr^RY%;qsC;4DS`|<&~ePX*_NG=*N zNQ&-VJ_iKWAYZV9I^OF}1~43R7S0AZ9f11imA$&icR!gphMc(8$iEJKNtKN^wViC| z|6~~l(mqu+*6=`dEpLFEeExRg4=P^V7$pe&Uzm7oYvpLy%VdLZo~eNPfmuU@&pe$9 zo_JMiC8^-QMdGnP_&vCn&OI4^Qo70CV8kR#v);v4_X#eo%)nGGnEfo}MP#T#Gof6Y z2Io6UogW14;Ef)tRp{0QOr;9lCcKHp`&3TpnaM+<2y%xUW-);J)01x5yYP>p5a|t0 z;{QKEczD?V2873@x*H(J*oRwLe6n=c$$5!k!$n0iUSmJD#S;~!Mu!4Gq6A| z#tmo6VR80+P(^nM(v{`PSK;R`L!R{Z^T{}rEM!gYpgc2f=ptVOLt?mf1JD0f!d(PZ zYd)Fc6u|a9tg%0uggx=(DZ0ahQSvNPz%h>N^ol8>|4-cAVrC>TA7U#J@IMCFt-EyC z;zn>f(;B#126t#(=JP2(ydFY}tVHL8Jr_KtxK0@OKVLm->%}xBbwb!5{4k_Brr#t? zv$>`O5*b8y?ZgF6ds4D^-GdS&7>wl-bz`jz81CzN-w!2rg8viU&O{RO*Mu$3r;unY z`C#5Y1%lYlt#9=VKo5=JJir*Sv}#qKsWU7a;at17!E+AtL)rZSej4OVvu-=>dZb%l zTaHD;1&gW+d3DojKJI>E54GrS#M|=P)S){Txpm3?4YhM?QHt0`#b+75>u47H{_f4# zDkIb75G78F7u0RrY3yT1Naxa&3;*cszl?m=_@5kgfdeo!wuuq<%nfJ$=)wX179Z-p zXv2)^?)$f%jAx2OZb_rKC=mnijL1#;qsN+m6{jqUn$81>t9<`@@qX<5bz%ewO6z0F zh@snlG)=_fQ=>L|XZ6FaD0)~11unOZTr=Uds^PPmvHX13bJO#0%Q|E{M~^zhZfTe9}moB|Vqo&mm#1eys(YTIqoeftak3i{_(6dQM z1f`COg0=Vnqs-@gkRC88y0dH^&apCFWwjWMZTdOnDLNYBX7RK9BX1O`?h+pv^*)=6 zAM=B4j3{bLk_K$g5z*_bSS|KOV&@=mHS7^!TsPT1<;tgMMV-a@Jid0POhKKBY79K| zcsxEe?FlcAx-bzAocC6upaO(I7-7EonsxMAL&DiAL{aQJuU>1l^TcsLMI9B=AYK~T zQX`trfexe!1k*$*J4cGt#-L1|k>%PkTHuK`qR9Ud-ObRf|MW52y%>fkMu3;!H6lg6 zp-H^N%YRRmKY`04^L-qJ6w|tdKi=6&1Ui8)KVPY475#nr62CSGOs$k=4F5V`Ayb52 z^-a>g%MFF`pE?A1f5@1kjwW}LUw&XPkrJU9KS)Ma2v$E9(s{Cu4|`!W z9?zZDgk#2T%nG@z|9;yL$!;0m^#i~%A_d!~ID1KLJ86-ETu*(}{*Rjx#WO>ihuF8A zf8|g95(ajhJ>U!*k4-{(bO~|3_5#F0Q_J6@x#tU&vdgDX1rAqGy0Gk8k5as7MR4d^YR0ioyd*Mp%H;$ z@6?~rKVA~gBOuhX@d3RrSx9#P4OKCnYdNIq6ptQbIK}s2CE&j@|56Gb3?di<$22|- z`%)@+A0cF4)5V)oxD)#?!TvCUT~yRC|L$2T23s2P$tD~ZQ~ZIII;cM4nKG-qZfBut z`ryxAXr*=jkmFjQ8K{ZYS@po{IOA*w7#_c2&u8A4f5H3&42GYTFbpO63h;XfHrQEP z`X>>khYMIxr%Jp1ff_iI5Y4chZZy}a(aK=O)hih=QURcYY`2?&21-4@Q3e%T>j1s) z4UnJCWgdION(uvR{#BBKRbltA9|+DqS`r(f#KB{B(5&YXW?IDVyVXZ``Y+%amP#YNLAx4-gr3y{BWiG6*vG@7}u1#!*n>S&}fFDv`qNvwb0E}WmsMvY`ikm`0Zw=X~ZZRKYi zK3>2iYr?1IaE7L$v8Mui)F_O%(rx-#wyO=wAK7;Dgvu5t@pD>u)J7!1?9^V}P?7Pu z)BDydDJF9JUOZk8KRfh(pI#v+%`VdC+Lq{~tqsmTH$7d8XwHpSPwwV?qa#xOu+{`7 ze>;TEUkA|6DAU6!O`RZa*p=7=zL>3Rj(jhlJDZ2B3~R#e4n8aP3u+|*x}t0Zz!H-A29a(=Z=3Xm~!a6RVgEC~3$bGb{E%Wa?e z5yUob=XjN=H%)(5UjfY_f|T2zqPg~c_GM;08cjz-=g{daY|_CNO$Zx!u8)9+%4A$J zir-3nGTFaA^x5jwrd%SeTDic))hf>smR%RzA@U)$`z4`ClM+hnw2k`GV8s*{scWS( z?W!i?5wtT;-B?;xCyGzD1JM`-B1|?&B>PAdek=PO=~2>M!nu3Dv0(t=)W47OdLw=# zF#|NKEoWA3bc5j-RtoJL5~`My39S^sa-`4wW3TVP@bl6P~r{DnCFJ<>3d7N9YP?@VE&fga?LIVAOT;Ib6R0#Lwp-~{bq1${E zlssn~h=Rv9naTZ7OC6++@NoaE3te6m03@>AYZ*oWvfYuFP`kTaPn!}3YHLqW6noUp z6E#_1)z~i#-ecxM&_cYsr zw9t?bB@}ZgAN@Xt59Km_DA8n3(H{@%Bq8M*hJ1SPexYZ5_B=_MnsE{m`rzc=QD3UL zXU*r|E)S~#hkI0mzViXCTwg(;)2{Nn1V+ngZ0|j=6~Jt`akUQ}!R}D>Y~5UPY@rVS z?WXLEMlQ|=@5Ai@Qyb&wqE`Tp=c-*!=6y}z2~LnSjUpkp5p0|k4J7`_0npcG5wQ`+ z;@$O&#k^An-L94fqJ@_eUyoRmx5}Wu1OS+jo%ow5Ne1aFw#L+APTKnhkr0>roV67< zn3wlLZrR_Aw){abYZT!?J>?lRT1c=x_f7)J-6YGtttLju?q`0xI}~=}*VnP<0Oe3$ zd-rH>{wxAd&;-%yhT+pDTIg~6WWBKCyGh6Dl=_BxKMxo8AlaKU%UA8lI3syYt2MU8 zSRdu(`sDJ6dFd7#EwxmGEJ-TG>??@xM{rYVMC<+_D#5=CI$Yr9wmy%u!h0lQWgF%> z8`_8jRuNQvYs`~iUwpU^dJBZObqGN^jZl;% zj^nVBYyD>E7aYwDoREJ*p(FPBUM;#O-X?q*IUSY-!lV}VxrB+%-#i809r5Qo7o#A<){%AflFcE*GMaqzm}5cTZ57wtq#147E`L*EDJcHW{Q6PPnXLhsf4UJSW; z-7d|(99b}g|HW^E!&;6rhh(tdEqU;@XUJ-4=ot>|-ANDHN7xW+im!}pL`Rp#bG9g> zOCiyDP*kKseDDW;o*0c4S{*6Wm4%wnpa!X^BzW;>ne@tAJ^ zi0#j-K}A@d8!s|)rjUKJ4M#2F@$mf(d!>={40iB~-OA1;TbjqGlS;xHuGE6t|kHmn;S61iLZcvp4R`BW}nQQJ!CMZ+e&>r4}a zRz(gp@N*|AB5bh9^t|LXf{jmAz^{BsCND&p#^>mlQfH>$-uEZrF;7n;Ko$T#v3UmA z>sU&n(?GMyhYZ9BVKyP}qN^RLNz~2ZB$62-5f;wFE!5-Z=Q5WF%L*_2Np{>~xq&+r zy#dtmv*nxMbT**kbtmC0zy5{@$}0_o0jaI_MLujTXX%VsUlh$2EsbBq=1a#8?%no@ zw(v$?bTmUe2g4Fz;4hu<*zIt@Zy0>1ZrAfWRL>~15+d(4lLRIKjAMy60X*3wWX5Fu zCq3%VxXdS?I6JirvX+MHjMuPCvX}%Sf^&<1y`avmDZuRa(zO(h#v6g>#$!7d1nUrK zOuovhto@$yh%dC-*l?+z2}`=p@04howqG07C~RcQ)ZO-~k-unt&53DU?rf<&rBt9l zO`zfTu9}*ZxrZx^N+@iJJb_oaGMKP%3`!(ISZ{ZUoOT(ckJ}*2;TPl#>3PV{>pN~< zAtOpl!yfXnKHO+stIa-1<5bOz7ux3k^Z)cY|5O& zBCR;J)o|)#IiAJ&o|N?%Ght>D(gZ!ZMsGMor}%x*uuDmTO^4dX&0Vb=tt)2Ei_-{% z5&#OBe0s@u=TY%PcM357n2_uL_@z%5Kc8l_ekrh8>MkhvRyB0Z=2W7#S|BKSf@ioo zk@?sZqp?LYuv;Q#Q%;zrLyJ-@gyri4;d80mYfK}-tXU>U00*i2Il>?Ste%AKe`+st zhFaSZf8WZ#>2mv04C39c9`WgYh52&JWvJ4nhdRO|P3Ew8p4=C9e6;B@&B$uik0hpE z4P!4S5N{7A!a!$xTz|J+$4nyz7^&gF|2|Cf?h(iY*j~||aJVxFquaO`5=N15v#j(W z(q_q8eecsyfS9>8cM`9W?X)M;NnO%tlzKC*AY{#6pNg~TwHHJrZnK+?x)`5>P{~-k z1$s)Zs*L&t&J7S+J-}U$D{cJKu$Q|$AS#shHAVtx16-(;&>zV$!g}q{wKiZSRcR4Z z;@j;2dmmq1;x-I7`R<;Dp@s@0Z0?)r0HwNPt>)=1`Dn=*6CE9r_^uS{s1)6%T`lXGbf58ewPaoZ}4|wvO58o4Nc12 zs?>r;eMrnylK-~s4GP|T_&ucjPGW^vxrW@p9#!n&zS1h(5iFgaHWaW7GkSJ{2l0p^ zY>IGMD(QgivnOSCjl~EuRA^~5EBu>aUV=y=tUihK0On{k(IeHgxPq62vAAEqq{0Id zXcW}L+Dnt)+4fO_*=_3!{(}L+!1Yr9!Dp&wwHR&R7CpL4KF*H?EJZ2>T$ge!4W=01 z;4x*zYK0tSIl{e;Io6if`=ci(^rCR3SOD)8xiB>3?VxQK{3=VH*mZI3r*e#U(ypgKi9JxZ)8=Wb=abTPRsx1El#=nMZcvUxvurh!2J#yQrCxE z2ml^J&?;_B$1mwJ=4Sn8s)=9J)R7+mMMko!r%UJYFb)r%BV6^qx# zdSYKk>%Ue@kT#o7HB1BZg&GamI`pD+p3@RLKmSv|hGmAH>>Yjvv;)Q4l7wBomG-`i zUzw1m4*JndVI^RY8*3Qp@Rd4JtC~HgU}OXBz=t9cUYT=eqbR;FR2kU)G1oMJ`e~7A zoW6<&Io8u_@s%=HUu624$_nwquN9q8vus@?NJ=KDQ2!kn5hm?E#+wpHcypH zJc;7#UbhI2@F#D@T!KCFT98z~8c4;p0zg6X>VYa;uCo zbDOI&cd7t{LOi3Q(>8u1Ip@dQg_7^{yLt%hLEv_AWQcry*%EIvRN7XPB&5n1S^W!y zfvcNLpE<)4pbCybxnB7HJ?Jex2-{s%f{gYQ>t39^_8+g8Q~MnX<_>spOPKvZgY~AZPo;Mg6@$A(UQU2-M_?bA+I-$?q zPFA2kEwhF7vU3LLw(-X0lK8sQ8uLi{pSMKK#fAc`vM+xMfxoV32DR* zU`NV<>1c|WoFR527Ea}eLK@!lDZq}DLW`?tv%pqwUxEz6_~5j4;mSbx=%0;0W9WNp z6NI+ls4=X06hF&=QJ#1|Q_%@PG3)l}BD3{yifd-~IXTp9J@(uL!I!|j8@dV7C?PCRWAj~csVwXXLX-85rms~G zW_U8PTO4;4Bzf(R^{8z79m8=6GeQ>|_GJONKj%gn1N)_`6Rf+n76PTd@uUQHl|z8( zHv0=nCf&v`n#0`6)|R1~-rXW%SF$z93mA;S;7OW1ZPO0qOvpp zS8Que>~~p;ph$L9pO2G4=Q>n)OVrGiTWwVLV7wVv<#s*yx&VL~q11wiEd2ClO^1fu zT_e8(Hn!;iI7}CJsAcaO*rOfhkc*`x_at$K_}6%vxE(XlnKwX z(PDTsLP_^|UfeSZq~A}7{IF0!v$U|iBI1SnAuY_4K1`_59Fm+A&o|>~m!YA1yfK$! z^w`|Szp7oPI&xF%4pG!>G%FY6=O35r9~<29DLe_$j&$&>w0_|UEQ^J@-fUUxQy|33 zSF?PdwEJbvOuIQiO(Z%YMza-+YZ3am=z{{_gK>&_62MPCXe?LU9j6 zWYFW_enV!9cZWe=&*yZvnKIh2F(H)%x62(HftJ}KwUMwijoKaL=`uNzsOPQ`fsq*R zTU0%Z-&gTjfqrz8Uh31C1Ag0!kE0+cXlIFnG@Q2SGZ6+?ak?Hl{8t$4uyIag*rRZ( zLIP0XhzbJD;+8>HTJ@G|18@V^e79BGGu`=o0C5??Wyx#5toq#uk6Z|3TVgDtX2}YO zEG_Nid1L2z5)#$9h6kx)W2i2|9R+4G$_h3^ws*3&)hl(T!pOrcQrOJ7?w!_LMDVQF zhTkS#u7)DQxr%jfjeuDbqw=jQ9$I+nKFxQPB49Q$r*%vs7L=$mW4@!5NA~Lvt0|*O zRUv(zwRlxEl{BSMOsp(gtvvG@;}xFrtOPdtX=&p9hlQ+1pNxx{Am&oGw6(>0D?a5a z?>20ykD*9Jz}zYuSpA#;Y#x(E6G{r|i%5}BYpR$j(OU`|rPIx5uMcxDumrFR05Y@j5HAZIR5ob zF+_DSx2mC&FkTBt9R`<*Vz$aVA5Blm%Z8MG7}(>+qJN*qSC=z z!DzMEKJ%Jn0=4tG=X>(XMkIFeU5ZL0V96S1z0*?h7GB7@RAf@`VJvxJ=hD8H2uDa9 z)pGVsNupLf#oeV{lYS_1IPJ5-@Tv7m@OFJdVoV-lKIHf@Ii)VdialXAB*!md?KD;| ziYTuzh;xMQV;VNZOk!m4>$uuc@H(Mqf!F)o0Rq{~1mHbq`Fuq<3q;S;S2B{eDXdb+ zg6xEl;-yuiYE19v=nEZKsFz75puGE9>HTZ#Vhq_u>a8|_tSR6coyijRv%O<*8V<84 z|6C$Jq4o3m%29z%KwxHT=tPjB?;m8%La2~9U+F&S#-9L7Kjko#(UxS~n@8~bL=QNb zXF))21nC-W89Av_Ih=+M$PNu!jwY|m;B$RWD+*Rwi9-3k6cBfMXyZ3o)BXd>6It`H zTm8m?mJqWBEsG7Q%_+h6`0bUK>-8B{SgA&r1h(tdsuT=0*8u-Xs!hilC_d-&lj4m_nki!PKfXHL<_|j zuIUWcxb&3{yuz$uS#E7s`FmYRDG9wgHxuapds)MLnFWa(xm^7-X88K0KMw(8ME#gD zoz&4j=_6BHl#I+MLZMeIE-pjb5vme@|55*ZS2nQt+M=Aw0PN0+WAsZe%@9D3<y3!aBL2KnU&Y(`9WTzuUii!5u_?QRO13a9uu%O(#>r~YiUuW z0(XVg39ct(2{Il_5u9|W=yq4dkico-bRhfQmwyy0_UaxYEKE;==NScsi*x9D|V_W!v6_ISJ$-r3v z#+=bwe{4?h-0BZvKbnhRVp)Qf8LotVRu;_DCtfG5`(;{R{9P2$;!YQbIihYzZ08Rz z=CB6s5Ino%Z|SGQ-M>T%)yv-S%5fY5B15v*?GJHt;_PJhEeE2wWcL+ZcbTymA^+rD zpIfpd1m}ud(N@+)xYeu+J8c;10|t%Sh3OXzuT`0hGsfm{J2 zYoy7cRN57w#y?zgAW@YRls8qN3JPqOqys;Ox(97g;wE4 zFDU@IHlF;`G}m!MnXu*BW@cXtuRs(<6B6jYHoWpAo||=HoX%EiiGpJ9q&H7`%JhV= zv0M4I{nl{LanZ-EA@ z9pnp=7TbWUnl>$wF)6pkbWGa)SC^N1?hzb^zcL29&So>R7>H)0XWd=JA%Y6Vk|g93 z8N--Mj{ap*BoimXsWCXhqvt)eCB;e z5sz`(J;WJs3z$1m&}U>HdNsR|KW8qB(=vwJQ+b~4x?S39#!OZg{sfq~56i<3MIK57 zs&OP4X9y^JRuWy+BLI}#(<_;rxs|Eo^o=>ixPRq+OC-2@{KC2?SE?ea)_ltO$JT3G zr9>toWQqP13z3YkW#IfwuZ8drvmCaah!pkp?zm1vJ2OAWusmadsPu^A%~!WdX9r4` zvB2LzPXd8SqimRf!O1aQw<26ev)RbD#D2GMNg+xbw4>wG(}A+}Se)f~glbd03zCFL zSQDU7KWSs_@#rJjAQ*-g2+5%$x!D}nh*$rJzh9)&>v^*CaMBZv+r=D(MdlnFRmQN& zRUbT=A5uS_AQ+otBh%d*2W)^-$H1#xI{qb%7g`sf?!L3a`jrQFlbG+b58Sm>q4etZ z8Ymot(>6MhTemO|_2gxgJE5Pm47u$wunZ}gkbLp}5@SP(WPiN91Vd=e)hVIL^Wu~b z6GQ(#_;_lERa~L!q#7-@4e?#o#J<mc~*Cp>|-`E7x14R3IUUVwv841Sh5vb08 zvAW5!{;OB!;!mh3tPThbD!Zr0igZS9!o=Qs(iN1_j}SgmnSd5_pLGoiNKeyw$JE9E z=5Q|5KzJTlXtz&q5O|_bKe1Pwk=c)#W{gxFOUhTP*Lz@$q(jQnGP=#R_E%ub0eEMa z?>C$$c&7tqQd0*qGMj7p4;~nd74?jA5bpVYuCQ8msdj2${i#Cn55AlJC%Je zBEd!vSPYm3e|q~IvoYptjkNpg@Hz|yj%0{^A4>8XA_$E{NV?zU zEK?$^iycnc7I5oYNsfq$THlWxx9NYNj8%Qa>5591fN87@IO)Q5O8uB#}Y z|29hg5OmlOaT~Vbz+%x*A0h-+UQEt zL*bp_WcKO-25~xymCi>`7l6_+pL62>bym@o*o7$Q*{h6+L-V6F+~(u?MYozL_2@-6iJi{&X`2`^#rAGFV- ziHn4c@7-D&C&YSMz|$~$yND(V=iyaOBigBr%H?r}r9aju42O8EQ=OuKE^IP-2YU!= z5*+nBaEt&J%zAG4A`r8T2?|4s?g`bMTS!)}NizH}Tj8`OiZPr{(Tp6ryQD4hs(5%Cv_sWp!x+0_wAB1U%qx9k$1k2WYHs+v@AA zrUnqR3lvj3{eIzASVAqX$`D~{toYvV&Cvf80*&qHog8@`%&}Uu%sy{7+HhlQIK3eb zOVS){!63}DtS~mzX;HgAwTeDo+7&=OhLeBkZ!6U9TtJveH)WJ|tl4^OJ+QC6hR2nj zqn7KrV{rKr+}#?U=6X@{F9OQmZvh4KNkGy0PlzI)mnWp?QDj0z&p~8sX@VB9r)EfF z6PvGFF-X_WL|hiFg5G*ujoZDs0u?J5$!Xue8F)7xAgI&R;uI)LWz(|Xy!4u|#)T1B zbng2}LbY+JIAiCR<8u(^J*=jF92o!ujU^vTMua_if`RT5*jsV0J=BKh*huHqjaMQD zOfqF{yeMJg-X%>YeOKnD!RNhtr@&W|L0yFIvU=@p|2Zh*&hr}e0o65(X77XI>%{!o ztiL}@V&qS4ui_J7DeM36?&C+#Q%dy3=dzT=n>5YMA0fCp^V#*i{JMnVOWTUsv%d#y z)7gQ5$DS}=bU^KyKT5CmeSCus(|Z1fCNNAeVT%mlL6CW#4Fq#|Y;@+@${;nvV>Qws zg27uWX}lS^t37>Mv;3bf&~Hw1&`QTb>+Exm#&XG{6d6VvWO=imbbXsI`wk`j*>LnD zlnp?w(0KUMN|sa?`eY6az-DfCw@J(wE<(Af-pRLOO{V|-oLFK0{k4mW|4(1r*tZ2C zpR$41^=xDM>Tv7*)%^A@urQzi0QO8(txEwR+olU3_eR;Y-!{WUt5Lo2zR^~Bp~{FL zUC@0?wPabLLg@%}T?n)h7pFA)W{MZXsa{@!tB0a1$zp*;UTqB{xoJtp;buKSkD!<^ zaN%u@)~wKl#8=Dr!mrnYrnQh?M2kEUl$KK1w|NYVY=c)dWkL0Tr+IhO(FIKL|69mvHgCjK1{>}gT8}ZYZQtathXy3fR zY8;aJT=bv5tn7*tAP$iM)UpPLaEqn?L|7yxdpbXcu7Be||ML&;^cVU4$=y4rNB;WD z{o~vH?Q=s>#cUuwO@nBkra{gBi5m#)G!+GeCxG-QsK00OpXyrav)`xQZyM{t{s(yV-G91z2|fr<5(Pjo z^(jjIPk$KF0ZhdH@;^|ZFDr*rx9Qy2)U0P;0>CSAKxPc9VP4AFEz+J3`@DBSr&;HQ zr_*RlrkIghwVK6&rd*|_n`0$xnFUl>L5_o_&$05cM;0ew&;&%hAFgR;%C&w4X+NEg zw9ZEY7X$tKyTFLaU}k3j8;!$OcI)-oloUx>&J>7xH5bU*C+d_Igy^!!th2sDuS%4JBdd6#H?xC3ee!#Go~Oid3s$Ujy*ZT)F; zckR|cF77`|%os|fT>0cU z*|rZ+lLARWUm5kp`lBA>C@aoYTM8G`-G4vO;7@ID_jCT!;ldsMbdnaO0@9NN^w*x? z$h{EgjxdL#D6|q;Jmp>WiHZtB(C1gXOv5PtprrwlR(T~ zqUQ|?;&m%-LW;06?~+|V(FOQXHye{i>1|GfMX+{MCQP%_k;0fDggla%KjLSFo(1Ya z3MYTt!zZMhHQ~$&w8i=>x;{>&j&|{)CpNW>2--sBs&KjM+TxXL)oh`{z9RKdAoBL^ zEx@r}`L=vY^zyQOz6N{JA5BOiZ>Zhm3)F;rhPJEu#F}d;(?f%zy}#;F>p4@g8cmqf z@w_C7cl85&df?MLvDRZ(H(s&qFBZUU{vFbBL^h{|8r5J^7`CTxv*WJ#kZX6>wD~cm zY;uDZnKRfEjS-k*c-4emjLLvVNZ1PRbMgajvO z{l4$>thGMtUPnXc z-mf!e$@eg!1v2~0*7qMiyENzo5B*F;EH_P6Hhor95`HmcEymI1Mh$eC?=yOllMvy5 zKKdGKai4y&{I_%VzmNTG_A-BNjI6)@O1w6MCABzWswW$*7vnomR5AE||TUqOWKLU_!C$P$@GuxV# z*U4V;R#W*Nwu4`|-$Yc7U=Q2{Ofc>Vc_SXyj%45dQ>R_}QT+j27{&T`a;#TA}=MMW*hew72Fhmk5< zroxxku`U7(UpwtFR9XK=KM`;o`I81tf!NjA4c~LQ2LjZhU+V*DUwj&rDKglx>@Th) z{3DT~S~0uTM4h#lss7xH0#(3Q|9`u*{{72bO8xWb;*bOew~o3Hg>B2eB+~dbXcq81 zf4{S)Bs`MSFpn?#bkS|f_+_z?Gh}|UPSopobSbd#p}6PeH+7;Po<{}KdKc3L6NO-Z zD*Z;)4_}*<9)d%04S1~ERB9T1(BvB}VnRTD=v=)a(6B7b^}9;kCE99#lJ!;hOZz?j zcCv=}CuairkD|c2VE|e6QS#*bN0TkDIR&voPEw zscJ{WJQm5B2q7op3BqQPJPC76AD4wWjhZ9@*`M2MdjxpkmgiWdJx1rF0HTCZV^ThJ;C=l&$H+!bOumeuta&MV`OFKoBk z0&kuB&t|E!P>a=lsjU=q6VLbNf4ar(wo_gflD)|1DPsL499Pd3M$h@?AGNSUJ^TP) zv!jmRcGQYeWwZSG&0vqrolnXcQ3ac9W?+RUJJ1Fq`^CH9inr!tciA5E+gBnBdI zs!EZ69jW(f!aqR8|EC+Xi}v9TeDZL;RQm}4_x$Q6jx_#K) z$@*s9oGfc<886~Y|0@+#I}bnHs*+5|krIK8Hn99aM48I!VLIgXvBa^5mxdEagM;pOk7c ze|iOgm=`?t(oSd>k5*eXXU~U}`Y)yi?u*^+8-IminP|2mh?Ny^RSNnqxD5Np>?p_q z>+rzqj!V{9=4s&wy_VSvzO$YApDiq=lt^!Hudwsy8jcP2ql8Taflh>DJ~;WW;N4^D z-E>omoWYFfFf2-&^|TM^MHp|o2U#{OGAAjw*V8~%`o@j!v2}LSiu2y@Q}!69_|4a7 zt}c?#CeK~0o@?J;#d^&OlX1ILPCGoUm$4M(^#1a}M0Y9QXa0)snA?vOFpLUtoUwVKUAv|fDW#*r+((C4pK>P(9j({GL!zyr37kVJ+Cy_U62 z9L9OkbA9U?jI;R`U*5B=-$IfX@!3TEddvLf%J9YefIjg4%qw6Pb$j7HhqgO&bB?O> z=w8Ys?(MI1DxX8z8uU4)Cf=ymFC0~XPTZ`W$qHJ7?V}#vaX{)E^fi%qJ~mU%`VS9@ zJmkA4hye8OQ(`Amy?o-knNQC?0LE?I(wKY}9i}crG3dLuzoU4hy)3&9F5<;OHhW=5 zR2KsC87cMQ767L6da-tO*XsZDDhwE5$y(d10iG z-@x{{05J9Z0+6L;`}%=`7Z38WQgZj`S9t2)I%$vo5|E1|K(iy6pX}OJ#uk@g@LJ- z)+mtYUG7AK4yTS=#LNRnwBWd@7}JNi>RW_+`!|h6hsmNVy!t7i%Nyk1Y}t0%dUwT2 zj{v}LhaUmf5R!(BIr4OXtss9Q7&docNWB6~ewfEz#~3+2xq1GfvkP{_paN#Vim z={Sgm5%_L&@R?X@>@Fqn+EDQtJen;an|!e_r;^Gn2lVTj$8V!TYrMt2VV-6_h^Otd zfCq=kP4Qk%0(!m5LHu&jk|l%5%Xcp^v&hdTo=ng z`cv4R?Z!zh%jgCID$XVVI{WnUi#F+~*i~tBnfDomtKS8KD1>ACH9QMj@Ay zr!@E3Od7cH{q&UT<>5>VzC=B)$^6gdw%ccu4FQ%hScz2?r4%fx3b-q*JaQG({DPjmbu?G40i zd9(1}A1oQ5XK2PRZdM*-TZWU!J&_n`9XhbwL=KF@5XHF6+*eed1Gw%`uYt60gTq8$ zD~DiyU-d~7m5}HIP&V#%#oMLiVRK$v1n%cS;B#H2h*JGJJGq(fuBd>z{{aO4K;Kn8 zKjc7t{uMxP+Kv|&t+SdpRd+Q~t|A&Yo^BjPl}zKQK3PTEl$SGQGb|2T=7@W>@{MHg zCjbaHxjMZUH_vpYjr;g9TerH*xS*-qqZHP7iykt2SE;o*l$I-QzR9s-#PX61s5vMY z+p9&l&;gb@1&l-G@sS~~b$u>P01>4~Etx+~C;uX@f$03%nwJ~v#J%-zgjF{uir+tK@$LG9iY}dcm$6eo~RCxXE ziRa+>FxIa}ISK`Wry%269|Wi@19vF3dlJ%sCs|?WJ9VA=$R25NV$=vUVya>Kcdx9S ztxL^7qC&SWd!O+T=pBLGrVhN^x9V*YTG7$+V1^Fi@6V}th^r^Sn8pmCJi0~rnqJNl z#Mu1)_M5J8Yq~{(c9%=Hb5>Zk-FW(Qwbd@XSkP00jN7gWxUrhly~CUPYzAddo(|@3 z0X#(zNky@Gj=k*NfhvXCL79PVo!%FtPET78@1lyfzYv@6r_IkD#HQBd#LS5+_zSLF zi2fip_rvI&6!Jlv0!;63SYDq#-kB`IA@97;`#4v=$(bb}BzqH&<39SHnIa6Ev=h{Q zf_y0Fbx3pfA{qidgISU)9dCF=p$br0fQ@VPYdpg>U)##~u?a(l38Jl+nL>E?U`1AR-#6z!zoN zIpD}h9m9pSBS3q7Ywj+1w}g6Ent`$EsR3RJME&eBA|;AWi^b=lJ@q4gP5fr>MB`2G zFTdj2Mgu}|PT?@rWP`)SI+o_+{%@VaLBH2k3yN;~p?9LU%lG1e6VTxuTQT@h? zYPR0*?}i>gB$h57%y~9#F6UbkAH=Td1HDf7o>>QlWVZSKOI=yz;z8Y|{>LSjK%(RT z@xFoN>!3o|&l&N(5_P}^{ZZHo`$2lmO9Sn}!(-@0bUX5K%C~T#$zBU$HZKJJf6tUAd+mR(@dU4*HmY=1t zXQDp)j@fE1qG@}q$RZQ`;=znuwaoq+_nj+zmDwD0%Cn03Nws0B^Xo^hcF%6FIESb= zxlTr|^ke<&yW4xyyo6dc1+PTyj#;$l`Ydp3%Ix(Gnw*H>h3}Aio=40?_Mro&!-gV8 zM&y_J5)97+on}oIhR%<&HJ>vMn>!6YrV_aFJL%3Fc9dpW%$X9B5A+4WgKz6wy~Sup zzP0{<39Q^W*oPgZud6Hyl*mjN`fxVA(yoV3mTBL2m50Y*&Gd8=1mhuZdO@ejvl2pi z^RWr6Er=_Yk5(5wZ|YzF8gK{aAd4{%Vzyr}oX*_pe$(lpI$gowJV8#Xc_~K6PgcKZ z;ppB1S$MAEC}5|451;ON@m~Kv)AuUi0MLeh+idS#cMfN79xhTzSsISMIn!vD5j zO2U0u+qqmqQrak2&fhu~CH{EkDdd0lrBW7l9PD(OPRq7}bt)PQbZ>)JI zu_XI!w#wk0|5ko@cBU=#5FsarMK++xbqf3KB5fUqc7W#x$bCe7wj4OopCmQe&YH#T z=6GPlwk;22ZODvx!e)?TT6juA9}@cXCfEON6n}pKGY||=sy@peX}@6dMq$?{p>(pp zFiR75+j#`ISt)Lw44N%Hzj#l(^V>BKFGTP2%drU=6vC`vwsz6_0l5n?iQ|C=-SVjX zKArOLxwy|-;a^|He!?PeF~t^B8y%C_7|xVC+33OMH_clIhK}|21vIKtG5J_B{z7fX z4;UoS58nAhoW@O-vasr*hEx1{k&|U!(Q*a(iK2%ItmRfh>x3Aq%hv$jAiB`lhRv#CTCF~MbbAsHuq82TzF_e1*615z9XI%&8tqkg0R z?U3FCWbq6Qg&7reyv*L}Hp^J+ap15Ea>hoXbp(Q8UtEfIbV=MAI0PzOo55J0q#t9D zTRe$*o_`unG$3xOdmpc@ed1!KmP3&`S%ZVc8JZ>R63tE z&B!FoUQ*C9F*b3UTuEtG$##BbwAL15JcaL01)o!2Tw0rS+k3Qp>JkG?mD)u00**6b zQBVDo#kfHb@`Z?D(y0AkyEo?dE;FF!7C{Ig;aO8Y@n{%j7stNZyn_H$<}etkHP?ykErq^1psg}}Ny?a6cGuy*+J-Kfip zDq8Qx*az5a)E>hElL>z=FZJY>uJ+mlJ7&0#Mg2{;nWK2PQ;V#DFNW{QF%!XBgE5D4 z>fxzY-X6#07MKO4SorK~jlhlxQXB#2)EU`C9stUusvNmP0DahOYD@n?+loczD#m$- zEzI>u*_Rg?W+AUpz*KFyijF$O=D6eHYaxHs!c=Vl1R#VwZ=J$7uV)6SYx>5Wph}*@ zsUa=21kajK3>-eS%?stx)7A+%+=O|b!|cAYB-6)H--iMtaFS=Pgx?K`e;k*WY{Wy1 zbEf4MwLx@o-~t73GOuZUH7ubO`?)N){2160?cVI&YN|8tM~ncbry{^#Ilxi#=VF5j zn$5jv;pEM{J(OBA@V z%3BX>ty=qbjPp(3!6`2_?ahHff>FOjPAg@O$3q#=BlqC||EOF-;F} zdjgxk!l5yzPx_I0ZzTjSur?wZAJ`i$w~gZ)LzyvLmpi|qy~9ZYE#h>y!~5bLw~W51 zR?|JqpV*sWs6r55mdzya;J5ezIH66pCNWIv)M}H@oVt8`zk#y<X&&c)TIW}`WR%VTjlAl&rCQ@7ir0{I52tSg6oT zx<3zC`AkCSt9-Bhsu#X!W2c^PM*23E_Vj2SUl@1yFmadPdf(skE@jXFURr{SJEq(! z*!kX}x5rhpx>5A4Q3kI>JMdv_$YIsb035(s^~(Y!C&mm@44ZDL(O~q&AYM z;`4IkqS}(U`=DiilF+v=QxmoXZ!V)q8Zc|#TZB`hNxg2+#_H$_BVGpp8lJcC_dZABVeWsfo;&y%vyv3s*jQkH1&qmEnyVzpu zqGtI!(13fyRub-yPTiU4>`ify>1}Dt)3FiY71H;03_fGD*+5z71ZRLlcT~;@TQ@F>f zw;Q>Th6d1GB+yZ2y7E~NRhVDVaKzgKXr|E|lC_s3{tJrOjQeCWp*QU$f*_t z1SbvJH4XDU0Mh+_(*HFlcg5Vvay!h!(mmTei5xlU9?-3}Gc-T-&EB z+Knfv`?CTy_N}e~Xv2<~1x|C05lK-t*{6_PA$&)mJqr=}(yyZDrC^9e`wTUlE=R1v zb-hNga#;bq8E$xU>=$YLb5vx&a&obslXZ++S)R{g;A8#~091H9HlTZtbRE|7S zUW=Z-n^U}PkKv6FrpePj;PLV}0zzA!eFDQ@B`GlqP0}@S z78l7Hv6mQO1&cLLj(S|dDf>GHm{iGEaypTVmqq#CdgO)J3?kut%Kz6EzwEr*H`saj z-sv8~)at9yc*Npae2XUwwOTd=CDw=%Cj4bYK&>=mht9KUf6j|a~2mni;EaVOAx<>y4GQw zVK}7tGm6^hi(N8t61=*<7DrXAn7`$W8XA{1l(=oxu^>l7B_oVW6m zH9LWWP8^e|g>AVT=#LfM1FvDmBHIm{pqs^-i?12T5jIu6_#X`YPX>b#}a|L50Qb>vfDiWr16TxJy#* ztl~urr5G3;7?DppwL)%Xd=%8T}->54M%+rG9 z63t7gtL36x+_}X#j%wvEu~XGUa*tB)0CBHyU&qjSYgz{t5vOxLJR_ zL3`5%s*XRxndwr!JbC&8h5iU;$7v{P2=(vkWIgl0zcv{FM~k((&a0Tl9s0oMTq`cC zAYLc(yOp*&8zdDXEA7pSiJM=*Q!UO-(gSUJ=1+iXgm)?xHFz?j=7t|uh&-E=vW49O z5MB^sJ0QAZIt^!uj|~2bfBSNPhi)bOygL2+98)>)kgY2d(Wx@8=sv3EOs3m2f47}0 zpt8h~(`ELC^-NdDLa|i;lKljU0@I48^X9Q@`o5}Nwf0Oc&N$!s1ib{PjIxgCCLo8; z)hvs8a}2U1ROU6n{0>EsQR&ngM+>-P=}=w(6Dwx=a!VtSDFZAK&)Q^C$VCia4emUw zgNHJ|bE7yIl?;k{Y;@Y-Ia+9{iL!@`!OSb)s~<{E^El_(Hx{!QEy|hK1}^g!Sj$;f zC8@qN`2Lm!oJ;M9LdW$eqCF8i`?bO4obD&=;mz#X5Y#yWU_*)e>Z|tiJmw?!<&AdV z;l}8g(GF4v!MsPe{}yili`Nf$dzKh+12((A*k%4`%i#sWUF1&PEzPLo;~Ek&fE=#k zy4g%#$=QN~TkyhuEDCh%5h+*~dl7dnvah22CtF?D)_fiP<%=>5%sfrG45!2PQO58v zDrhmu!f3?x6dg|9WN~={M4geVt`?|0s*Jb%R&O3Lk@0z>MFR=^&8+S?_BH~$)vd6vakQ#G{%S+$L41?+}EfZo_ zaDTmL5m)d7rJEOrgnDKzwPnLr$);R!{Jrx$9J#nY&UsaV1feF^z1A=+$x0BKcm%@zxnV+MKmyvOU)Xu(9@YeW95?GKK9*@-ch7;fE6j^x*=I9t1Y}1J=lo!ELMnpIcb)KQtx-)^#;So8PtTQw77OxqQ*N@XjsU&| zq)M#Lb`8MhqDFm{6Z&o*+#R(>+n>X4RfiWRt>55rK(*>ja!XBR;EWSluLYwvI@n-e z95*)NFYxI2#@bDS`|85mcZv2Md-Vn_XBDWevH~cYuB(VQbu6;Z1l=QCR#~PaTj4m= zbhS^LrlnsReN!)$`Jh?e^9XZUqw<`(i{BA#x}DPgnkJsz(z1l_4f$p>vxVwd%TrDx zE?)!!OiNMBx*L?U#kwncI)=8wi>MDOvG((h)QNtZwN=0iAGMKs?pI$!(Am*~7^*gH zi+@&P_D`WHwM^j=(Z)lz z*&2_hV#8YTw0|Kq^#O#YxWt2v(`g3R4M1o*|3PRT#*|UviJ)T}9fIUi?7gs!Hf$6o zOQ)wXJY70Xavt}RQ`7IobG8rgF?RMgs^tK!K_k3)^CtgR#*k(AJ%`tZKTI_#Fa+_t zN43pcu1Nsy!6?;mWvILq zr!2JwT8~A&?6JBazWQdDZ&JVUUVp`XLdp+!CEN+fM^56%4JQ0lHMou+N&`KYr<`%) z?FLYtN}kMkHLE=~**{)sv9;^-TfDBl)S4*w<+Z5qamPEJi4B3~=%Y)#q5T|Ih0JTt z%}b1)-*>o*G`lacf@gi1s4#Y9y1+&?k!k-5Y3iN_hj}nZEPKJQX3Q?_hyR0fCpC5{ zi3^WLE{d>wi=}%Sa5BGG;BpcNK%G6rO{S~ilmMr5KnN4zVL6IK(T)r=r+H6iuZ1G& zYBzHl`;!#qXgJ&ZBXtgtpEsaVNT^WXXV!5e6>SgK6zruzm40iM%9G^z-JR^^ArdD~ z0S9?!h5lbDWPs5?D&uJnQWl+Bz$y7LbJtTU*}W-{jS)rdd&K1=7>cxEdi#%lyd_2g&oriK z8O84*&DO(~IIW5eF6;TU!SWerKhEf5eEWTzE|D5#b?d(H*}l&86LOze>&+I)66C*7Q4tq0*plRe;P4;c$v?U^sx^3j|G2kSTs#ds z-`wI<@by~is=_}bk`{tuKf|IF{g`X$6{}{94C|m`!^=1a+>vVTc$dQV6;7J2@Oclz zIt5GY&@bb|&)o<_zNevj^liHPO#|8F+A~1)9rO%_<~`%7Fs4C?@7Fk4iv%@`7+9cr zd5v|`rsPo3Woh-LgV{;;3l=AYsjyM%5i8b!72#P%j zagpc+?ilAw2ShxNp_!^=0w*;R70DEXTLFtfPr|?E~ZmsXm6j_~W z*%bCIj!~eMeR+1(Qx9;XMG$EYAIa`_k(?7@kB5;%{N&Krrmuj{C;r)k?)iE#_{cA$ zyzMT#{Try7a`k{adXe|z$NpDHV!u_W{_aX=F#Z$S?q&xN%~7`>R4%2^M){h2XfTO7 zM7zC%>icVKGRYkgv=sot=kC0zKzNmkMN`V^1p*RyRFAwRp)@2R;xHk=S3xL?5cwzr zDuW;{;5hm#1mQam(~7dCEM<-_Ro`aA+a164W2@CNLn)649H8O50-E)E3Z%eO?2g^1et zFqnWO=9Yg_OGuif&kj+^QTek)A7KZa0KaaM&ly4%Ee#GNwQ*Hbfj}K+`W=FGJ7%7F zS7@T0M0gM7$TA@uHoxEh*9`!)9=UxmnA3;gTrKe?c?tzPd+kmRg*|J^31pELkrw61 z*&VGL5Q>?8iEYD(F7a#mh?;itU$JvuDz*p6@)bpspqaT+&YV{)0&35+SUrGTf_l1i zG|v>hGqe~4EXWxw<5a{iVfNhTrjH(Cm>Vt7t4?Cx_B_eEk|8!P&o5teARk|jy0Orn zh+nSBVCM_{>R*M-7qL5;w6ER(uA+uvj~diGCN-0EF`FM&d-#P|$8@xq$J;t3%qA^2 zcBemB;~?&&f+=7s-%80l>2A;1z;fX8lhG%ljVf`AfFw%tNw%>-ltlDo%`a-#Cw_W^ zuzpoo9+R4yNjmtkvfjT%r@yVY<}48_k<5+-{AN2a=qPwakC2<>fhy>RV6X0E^vgq0 zAaqQ=y^sPwcC@(R*Dw?1!l9hJXEYhzcmDCI<3Wr921^LPr=V2MY53J8HUUb<2qk86#)*~5#YP$VN>*zp{eG(sc*UV8XQN;Q(;U}OBu^aZ- zP$(&_?bbj1FSE1RJGP%m5P`9CB^(>7GNam>wO;BxfAv*V?c%UHNJ@!qF7A^eb z^OFw?l{Nx&UV^d!rjXk8GWb)RUpeZqE6RB*jD#i6^e*7UY%Cy`VPy+&48Cda*b0!b zj8TWl2^x|OBTfz(@>fe%$w)3NG(&5=+B-tvd~{4SUa<$r6us09R;n1O~zJT;lAPSj$rK(p^4rk3)7 zxN$2xe*QyG6Q{!Fa|)-0W`{?6hY+9$Gh%-qOV>({q30px{NT4%VhpNv0F4$K^VS%UsQ6=qTA0K0LqLs2 zC9?cIyco&ld&vd~3JZZi3$>Al&w)CN1XLRtM)bVdcEKx2LZ`TAi!#o|oH{er7X+-m zMS9c|O|qUVpK_^IKZF@aITZ=*oqs{Hv(w$yBGcNnfkS(~|JZw@@7CNShI#c0i%k9( z0Fj%Ea-FxHB8UFGDbH{#{7<<{GU<=$`_W%q9YEV(S*t?sc7qrHE4LRXUNI(Xx%C(Z>* zJEDmy4tok)$`zQ%O5H;v?$N^Q*tIVj9tDc*tfU3sJVeMJoXbeqeCS~l0H&2R|GV96 z)1^pGXpTgiez@-;^jE&G9=&eZ5h4K%;2_Q>g;tFO5aW=WSGH9hss|iq9nJ+aAi4-( zgD8GlbTOXKj0QJ*K$AVQA>`uw*Jc5GongxWWg{KIcrQ4>|2)eUfp&ap|8b6Gmi#@P zTu#YD89|Hf)4Q|{sf^Dkv1N6LPY2_A)*f@yMd(pd095a1?-C>nSlhO+Egmr@*H?As z{WhF4IqWpXzmg4CucA7$%AD24>>$An-}D8xq{f)3Zz}M8)Nn89+iQM>Qb&dFS_oYWw#@T+{fVtS>{OfuSv ztYjEEHxPe*R+50-(|Z>6c@m^3U`$1??-H6E)uoWj}zJ=tBf&i}hWE zMKD*5sAO%YeTm73+D|aUb@A#|3=7p2SLS|Fu@my20Y2DY(VR_#t2DOkjr_;6_7e`~ z!fT|1ISVJ1G6{D`-Y78{f}SsMoPgJ?Yp!Ah_gt~HQ}AsEzl8l0JX+EEPHW5~XCIc7-N=e8O?WKnkogkZ^u>>I z1%*@|S!OnPfuK)^ACpG|^wR>6Gy*S8N#OCyXCkWs2ch-FE|+k%`U4Kw4{1wylyQLeG*X)75S+hbhag!Uny+ z1Ns^;Axt+I$gh$Nfm)h!<(6Q?L{C;L2h#l7Eb^x4Lj(;%I!68Oy7V=+zN=6=agIdC zamO5zV*rI5X(~w`dB#hy&3Tp{RKpc8M=d5D4`fr}G*{v-g3`6l5&rsVph&mdkzW-P zX=fccD(y2mzL|{pY+=OgfKz$+aWh`emG4HMhlOClH)3D^beT?=D&`?bLx=8Ud_C=@x7`W$)@EmA{8LmqGZt*f+9nA z2ml*^AVbfO_jl1S!tUsob@s1=$~p~2J54r{mihWvLG>fTA?9k}s`j zpNlXJ%-j0EjaXE7t`GPp)Kui-6q#zAa}9=f$_-O4;#0Ah>d#cUW)i`8T5hT}P(@w4!c;-7)D){=fI_KQqaGpR<>w@o(G037lS0fFvt2eoaNNaPJ(^R%S`+9`r z1Z++YV0r%fCzfYpX$p(`@)9DPuejpAY2?<}`R>MR2QXg}zN6@RVLO*}EOA!5tpP=d zKgKF}`>RifiCyFfCMNRPzEQHG>oJCSXWSbU*B8^!|3UP$eLC>^=~XMy+&jM{QIi=D zJF~sHa|_GdxoDo8KvB?~Ugdbk8MU8j)DgZlaJqnQ)#-#$qkob2-!8dvK*Zy?<1+x} z)TZ(8cPUJf8>Vm&$6k;aWWTA?xQ7?ZSE{LYcAy;@Bc|kq!|OEq4ysz9q|6+$Nd$=Z z*kMjo9i|m^b&D3D5sp@P)I{e4=L}=9ZvP{SEQ*5AO1l@3K78=V6DrTHTy*__nEz}S zQ=M(=`>o}U=APcuFn+90zYqze;3W3x*lyO;V)zLI8)ZMFdfe{7b$@j!Sp*C!)A_-9 zH*6`_EPn19&?S> zd6oeQphqpT437tPe%iabl+#kCcn?9F*G-l!hOa~+O33e3C)$qQEt;%n=z08YVqswa zGXn?w!Io{`NWGaY`lh_(C?|=00&F(;P^1O$;EPWp27Zd)d0LZ6xRZfl@TYXcppdlo zu^ffTW)yA9`?5M&!3|w@5vTW2_4xvh-+#RI{m1DD_>Y%>4Q$v&7h!#29BzYNpD#`o z!Cd9|?{~5o9w8hKkle$0pi<{l1I@V0AQY{BgEL)b)YWFD#+7g+e49*!E>KdhAc#uT zGvFg6raDVaa_e{F6wSZCPhBUU<_EsNtdl=`{j&KrJHC_=VEY4X%Atbho`RYNg8rd+ z=8>O?t)QXbwTpexQP}|lZJkZNy69oHElJj z)60y*n4#M-wytV~m?Z-Cq=pT@AH$q#d4PZmaooL>R_xoVwp)QHxU*6i@Ucd2Mo z101`0OB{6DGj3ng?>Oz3Y$C6V9S>+vp%(Gu*VLsepW6%B0vlb&9`zH97OO6x6{#&} zoFA_byv!fVM0h2$rXx5i1U@fy&cW|Li*ORplk7povliy>xr7!N7<%8H30zF`L<7Ga z%1VF0E6lqs50g9Mk47Un%%Ws#_`?shH}ohr{|Sm+W%*&4w(v5A9)9-<$ZkR++QYGT zH2}!ZvyvX;Kfq5^i!Ga0PkMLpaVIv}_=kGn|3H0)GWq>Kpgt|0jrN)s;dORv0ZNb} zqrKqbtE5r;V`=jR!?F8CV5co>l6s;X0s`7*bKUWX{f-&s$%P-o015UJP#=vVTbuB1 zw`UgP@D5<#D)>twZ#swx|JoQLFCu$k-%sJ4l5HUL80OhZKNcv+(}@7Ym{)sg3*Q_p z%ri>V{jbDNc~M{3&A~%*>~F-+{p(f3G(|bjrz;{s45S#{9(ob5TZW^hI5fE1jxL7a zC#6;0Qvj`UBi?wIO{yi6%3;uXu=1_e?vf6uh4*f|rq6qnhV$3gZK}Xa#QXMpQtkU` zy61py%iC9Nz783*)JywX?+{ej`S?Wl@E}ZwEu~`bqk_QS_%Hw3IfecA@0;m%5_~BB zrO_!MJ}L)?pWYf8w=PD9B3wr@RclA<#s+!oK+?9YDr+|r-6Z76`@pPhW;Xz$Oyv99 zd4@JL57r5_-G3BY0p&H&=?}liE)Xnu)`Hh_dLS_9nO?3_@q_3RfQ|fRy@wX5H^1#x zVHr057e4eM9s06D3i)^Wt6ac@qW-T`P~BGnyDfch9vc`ixP*K^G)2!R5XDt_QiNH8 zr7k;&t2h&?FZeX%`Vg6o?#0m-b`@!z{d9FCeFNj>?p{xh9!BO)#AjmaB{%XQ#)|iV zA@n1G*!Vk-wsY+L#!;1fHu-Bo;kOUwV_&+`I*d6C3Bwfc_Xi`RG9y){jm1QAgi|l9 z#_X>4yN&{Q{@iu@odkGKjzf?=N%toU3%_IvHnoMDBEo@s8T^p(SKrc(h_I%9T+pxr z5bHhlz|UyEhXM1&Z~Eto{9j)`3!{$Y)22WH@LEH(A1>ttGV4r5or2MflgBTG{AtDIQ?$6 zMvrNxetR|n_T84^6~c&MNw3@0#G5$fggC%wr@ZLX7WJ1 zH6qn@4!qw~7mHous@8q>ZSiHP=7*h$I3BNcP)u{r;->u+RK(A4rhC|yfp!K#KR;d> znOMNgn%NhbPR@vlZfdgJ@V6+E^UY_xzXdu)_Yj*}H*HsKb#u zZMRG_|H*Hl*nao@^n}%U)^@GasL`~V-vee0T_3Og^zK$dr-r!7K5T`|`DKE3QU5q+ zPaJ+08!bB1Tp=DGtDyy^iv-2ymdIHlo!Y{~TADS;6o@gRuV02+G|dbe-0_A!sWJ7) z+lKHN{9?f?#UNysy}iO*j#Sq3D$sV;iPfCp=RRN;^;qfy--KwT3RtOr`Z!FGu~an; z6w~}8G4!ajjy;UG-0ieC(i#jPa8XaG)is@c^l2RnSdhq!qvtI;`;zn&yZpk_hkM=Dx;#4qyZoNH4HM z57InQ0o~M;E?N}u_8<^su2EhLK}?RoUosConEK(#vP%y4e!;5@{asf$izTBLVOK2u zKGZ&qW_F)G00BH;*8M4u6$n4I)!EEA^00Z2dg(wrVh8SuHD$tD2U~m|5tTj55f+;9 zPNOmRy~bt_dc|TY(XjQ%Rj^l2#2iU&=YUcBwBjaCWQGWh7S!mtPSNsbw zUfWVnz+LCo6LS2Po1K$Kg`11fU9pX?2NO88ZrI6q9Xtc3P$0bKTz}xBBrp%$!7_^Z z;{O#YflKv2nWLKfvtTZ<+f!coL5SO~HT_(N4cZ_hkXD^6HI^ggxe0z)>WU^D%rqxA zyWUBAxoq-LwQHfsE~$O|-Sbsx<@Tg^w6oTJR+o&|U0>0vJ`!ahg+oIN>m!--a=Zkr z=!7kAB$?^I$fMC&IXl1YW!DFOdVdUAV*ZOfI(_mNdGs|Z!`Kt#Fy7)30cj^szeON} zYGq|c)dj+Z&_|+rlww@p-@);t!~M9V`wQf91l-$=E)mnPPd(HF(growJ>{}Q^{W)Pkup%XD0&h}yKLcCZ-d&1;Gy%oEof>QS)x*g z5Ql;u9xHw~X>q~aKmsy6xDs_6a>RIvx>k$NR;x#1bD50oI1m!yLIkVc2)dl&g3%Xb z4nyt$cd6R-WJF9h8gBm?NqXdk&DtS)3Lw#}D*5%q*`Jzze1Xi_rD$r|`g9=mizJZg z;)V;tC~J9h4C-^DpK@w{CtGJ5t<7wsv;yAIxU9oj+_}36gL6ndanywmTYjY{_0&OL zPVgVpUF0>!xKzs2J$$Ae!%1@A)aGf5%k>*lyu>YX8NA$=GB(cUQkxu901)BTW{mBq z=n-vf#-q@dkD)#0<3%ct_22D&rvt%?=`87x#;wh#oS;0wT(+TzlEFgXD&a)=P)Rf* z`@i5uO=J<(v#ivQiY$*$j)`U5Ei z$m}_RQ+XPtq5I%o`?q=}dr_}h#qW8F(*SP`(=ZzVw$=Kp#aBXmp-CgdFSwU7T{41S z8ozriS7!Om6U8nGK-5xIjml|;8g16v9{j2%QW@;RXnN{AG6j%DWDOHNfZi5obiajc zKTaYM)=8qtY9g8e=)n|?nSyf3~A&U)qcnxBlPa zdLD?v&4v=swJ(H-N%<{CCPdiHahcDy`4WDLJl5AIWt|IR8lHA7)*2iO)omHuD--xC zWkG+~DIeiKS!O8nGrE;jVF?r>H#Yb8|t_?+nsvt7VsB= zR5I@WM*=D0SA<}1a;-``fLCdeYf{{cQ8D1yq&0(fN@I8LunJ*&(LBqwJ^GE_e&&?2 z8n4S7cU^9x^o{r`j%Xk@rRby)k~qZfwhqtFY@QvUS9C;_?Nf!#AA5a2bComsRBr6; zn%z=FwKq?JzTmpdqAQyPMTsxN&5`>25FxWG_?HpH;$XdM!F)*mjvl1D>h0toAFts+ z8Gmy)ES?~CU2vag$G~NXUlVW;OS-{(N3*yjRTv}q`Z(WYIEyb*uDB{^Ms5x@9`>*V z#k*P2M|(uO2K!a@;n`gGx#kSvDDlT?CYMuorF4Bn&ayW$N`4gz$4ry>=KJ})dGVVF0 zpX_i`l%)wHs^C9)PE7~**_fud26Zt+14if3Ij@8MZI-OM@7U@~Q4zaEBoRm4`IrYn zf2M_oX?rMOg!8^E(ktzI@4_BI5wOXaL#=2Ue?X6Wc8ynW5k$`TqaiYJouV4}|Aeqs z;6#j0wCpyC=P!xhb>>5R@d%iGe_$qkP|vyc67EjEXVAuMp{ISK z9iTm^S1SQdr6d2d%fP^>ul=sAIzGB=e) z57bpboogf&$Q4&B%O))~rrKC7BFaI~CKbqeh>Zv%K8=gI5BkBI2_mXF?ml#aZZ14r z^n-g!5lAMpG!SZz)rnCp2Ohn_#B4F=zT!kvi> zxJcXYjM<>vKTQ5J%}%oopv}rT^Z>Zo|0miaIQS~|i(N)kty$+&wnNyqGG`cYpf(w&$O7);W7E7b_~yle^5^_slgj*I4z^g~dH1yXys;oS}N`dA~ZR z*2Q!VU@&C7+FQlLOz`fE77%z4ZrDmL_H+mF8aOQpXlNIG_`eFHeKl~>)#1XUx5Kk(F&2*K$g06+v~6zTD>&BSE$6D_3b{b z2>ScQ;HeS-(>Y2QluJwp@bDGm8x1(s z(%4oG#dnxa|36qv&2ML{$!|)x!sVjnR~xAr<7ZRT$U}Y%1yBqSb+An{oRDor%$Az* zX~lNyLDX7T*3j;$qRC`YfPf0!mNscMB%}jwzYyMfm*R)u;|UvS#|gz`a+4mLuex>G z$!Q}eThVIijw;Tvw9G*(^-|sK8oN4+*aHMMr4jYCl9AK@4)v)IeFl~{?|qD>y?W<~ z;hL)9O~jfkVl7g{JT~|7-F}qe!O7f{HoT`|&YM|m6}QjGdfkW}o;!HNER3A& zHj)ar0)wpzt>jb=8VK^D%*{aQ&K-tZkJ?EeBkKZ>9<0lpM5n=Z$W#2`D2i#5wBW0@ z$f|M{VePLpc^YDriz-Ub)~+Ebthw4hv5x|2>5GtTL^;+t5wi!7vKo1PVcWr-ttIr> zm&1*haFZtF8qV;h@F@fL+#-xe+6shg;XsBz+`pxp;)BQ!2eEwbjYN_1=o$(SuIL>0 z{Wk=4L236-rcGdJK~TL=UxlH9d7GX_F?v77$~p1tM(f!Lf>%CBPTjxN z230f@i9_9=W(^=QZXhs!z3a*gDacOd(3dmJ0q(GrgDhEG zBLzvN7QA^zkLsNDPM;DZiv6hAt@1~eudL8v5$)5S@{pj*lIJ!|P zh}6>o(sjA4xNwZ()NtBrNg&C)`|Q(B+F_6R@e%D>&bBqV%@!l*?GwzcYUQHm?@H!D z`UBquk`3Xml_@I^tS)}hX~^TeDe6)3k7WbaZWl`P)keqT4o!{Bv5|i+=O1hOw;!*V zj^><~*Q7?r93H0w0DDG9U%cXrd%?asfDqdw^UWHxCBs_x`++sW!|9Zwf(qNBPe6pY|_``XoGzwCe zQ&!mLB;~QPGfPE?RJn`vpKRM~TSK~>gn-FcnTL^ZFdpBn)RXMv!X=&ea1rd{%6@zJ zbRg$a$(^|U{nJM7pJH@4WhQ`QvLavR2o*Yo#`fo8bSPSPj=maX;_09yoR7ayq+9+% z!YlVRi{J7oclu()9H~j=rkC`(H@^tm`G{@y0>z=FM`p6Ys?z(NeI-kP>ie+vvL|6eF zMVD1(?D*QtXDp<^?&ne!@4(|;4bc!_lCgzUupAO&p^lVVXotp1ofBVzxqB3Du6`Q<}H7t91UN~-$wG+X`B*fQis+l z)fP|^yiKqF^4E?Us!-7}az9+22R2_Lz~^Pc*slhs$b07x#ftOOqef$H>t6hqg#Ftm zM>$|cfVYu1cEelluLdo zJ&L@q6mQY5(9{GsW}f}UeWA!ZdF1ePyz-x%j;RzRrg)ZL?(^Mx#as9VhSwAx{7vV6 z^?ek9OjO^+nwclKsiI7Ob@&9nWjmS!!elT_5H9@>a>aCqH~Ebx66%BM+|u|)q3`x|wr*M$R1_BNpYe%P-dR_>NMDFwBlZ2W z*+(pyaJfnr|FT!Ud&6;MX0w|m40>~cUkA$W`6wsc%-Y=};WVrLhP#}B^1sWLw?gP{CVh7iwy~82NJT^L@sc4UUm}t647ge_gkLMz?E`SBLSdy8Wgf zujz1a6UgA{AVypiK%#JV)w==)c%CxT07K+Q2zm2O!K{3HyhxO+{`#@n#iE1u)-dDt zoHU51-uP$qO3}1i2upQ`RyIV~7)w>=gJ3Lk$G?zrT#k#(NxrNi;Z^gn|6pt{&3Ao; zZgn;-M`;Af2$%^1+iN%;tCZnyWl}=a*-?zWq>83w+n&Ho#jyNl??mTmp zdt!fjq-UUqX0YzE(`55ai-G)4#)$?%%SPce$bRWMPv77fP^nvO*_q5IuA*K*?-B1ikbar+0Y>-#_k?M1EY<0}V z=)=9@p0#}Dt!#i2=TlGyBo64O`luM2au+qZuci{yDW2&rcyQ^&_{c5retwzBcRPjS z_8jvls}g(awY>>|`ez`^z-O!qe+_5afCM_LMFk&MnABn2%D$IBcZ->lW~6}|7yRd~ zvZdzI`#0JBtBAw_hXin|82%|1Kzn<|flRp4jIigk-c-58>YNmuL!4R?e7UpEWAWNO z?yFaUy;hq+qzOtWLhsrzi}`$h&o1+H1h!VCLwZ!dl|y;R{}f}XX3#~zC>U;1piL+V zMa)EeH7*+9N`0fYsX>hD6rY4fs8v34M$qX{Iztpr|_B zFy)@fNcX)m@Bt#u-tm9R_!NE4C~1&NNhdU3XQyBYz?P-&P9_S(s{%5=*Q->l)l8d#mS z_PxN!a{an~5uOo1QP+~_dn*fpQq#^VGXkzb18}!(tNSUb&RmT>?nS@DHPho`T!4tW zRNH@Yx!9r$=t<4F+5NWE-8YQ&Pg6sD$HH0v*FpjMLnt&4Z@siiOFH#vz%LUgVmxpc zgBiI#GV=i*gvH9I80SGX$%Lh^2paO}wXP`sw3F;&`s!)&m)#bgox-aQWGx5X%e0E1 ztTHRojImPVM@xZ|x>hb=Qsysz=(yB?qhT?=-Ut%yU94!q`5%A^dT+bs>1Vy!HWnN! z`Od|z;1pfBM#A}%Q`~|1T?X6^z5uVZL+1i#wNOCrx2KZf5qH$>Lhl>$CwEO01p)lJ zx$G{!gYP8$zRsnSS7g!=9IzrK%k-xH&kv?efdf)%&WnLv>=8h)dw!30JyOs>;)ooh zc%Q&$f3YCS{e7}`8@EHp{s&NBH5`kQ_nJS zm3kmkkGg?~wl81+`qfCZ$(YF$L5h1YN^nvLO5NVosmNQcj%LSJZy-}ytY`kzYpUOt4Nz@2oA4vD?l$(*R-&> zg8)K~TwlLNIL9t$8Lz{=^xBUY%32_nvNn4;PjlYn*^-AxsZ|$7=0%2^Ti&Wks2}0v zu$Wf%b<2s)h^p(l5S?t?^Dq7v(H>L1w~OZ$4r5?4p)c)K=?I2G)sGAOOKG zZu*Jy)^ezr9n0VcLhrkOv%wpOnXOqcLlu(u#;QJ@`b_J^uZg26d+_YbRUZG{z>}P4 zR4gJ5I@CQHgEn_dv+^^rtW-V^XoQegAga|5b;WzW;H>o(9g+R|tLTw_4ftpw@7n^F4_b zB7topP4z$_>qh{&=zNfUvDd@PY+Ij0(V+H)6mC+t_%nw@%xu>^%$KH@5fHeG!WQsah{!kTQ?o&h+eAd-(cfV5KVCcK|*(pY|~v2{FE+x z6!~IFd9%K&&Bvl%Uu}>IPMtB7U#+T*{|1wC=~#X~1=}IC3|O47XzkA3v`0y#{CL-v zwq77+vfU*t`3RsoH*|ACN&uPDdJWH0bczzX&xu)^c8+wmKT)!)1qb-7{t!@ICf+QD z`(&_pB8z;wU>gQdvmS4^#Ha7`_&TEatJ4-0Up_H=(@rLWONb!;dDo~9ezhH{L}+(f zwdzM);#Xla-!=ve-h`e6v3y5F8cYE9HGYr?O(MeGnC^z@KdE{s69RReH!B;Q z-ORJTUwT<(wkx?-)Lswp%*}ytpe0kIUhXrQPL8*qv|<*cr@gsj@}euG}YdvtW&UT*Jd(-@WUbzek#4>w7rU_o z-?j{07Yp2h&sI%txUHhVbQX&`2AQ6vr#EYnOJ0dRg+7RIONy_z8(eBe%QIpyYkU1| z#3TmN^-tk-bAC7@ayD1A>4|;P1L|dReC~i~X2YU?n*w;D@Z*C}R!K_6>crc3FGKRX z23o)gv$vEBoEsk1-~PLITl0wu;QGA5-mH?4z9EGLd&mWTyxjQ^;7l|Rt$MfqwnG|Q zuV<@Q@0qddY|>Dp$0!VFGi1RKWi&jORVU#U1g*isXdh+JaewTHYU}EhjDt?<+L8=5 z07E&NQ;Jwwe)Gj;wY&XrGnf9&SC7uv#w#vgayQ@_!q(X{H)PU}8)@aJSLi&bhIs7h zqa4mI2E>(`%ufziLGpc{3N+xhydTWjr6zV3s27w3KTLA}VLfLMj(7grWB%htv~un_QWIA+W^kM}-%|7o zG)i?&KS)9nR;{Rwz^C-7*+bZ9@&nt?aYENnTzoYfLWis{_9m8HBVn+U%{Bv>F$u-p zaR3P0;eo?{RON#X0u>KE>s@K>wxMdVg*JO>n(4B%`S*LP2}? zQ;MS>$K?q%Ab^?akn(18S3GEoow++1=y5p(1gAdk0FY+neDg6;JN^{+Nt@ZH>dYlc z-83n-6m|emw)DY30?Z%50_mouwRK$KPxCV=k1IAR*9?(gjdz_#qecePH`6u=BDbf# z@bc@Bpt<<>T}3V`u$|3C0_ce^723D}H>!FYG)~+CsAt~M(~{g-mQ72JogGnhr=Qv1 zMBzVn(mb}EDDy*}YbSC{aP!dZ?Q=+YcDVa3WVmw}z(ypsuQTJXyQb8g>&qtqEJdW3 zi%3}c{nqzk9h3N%y&!jQk8!pWj>z!4^y$c69o*C+EGMwPiPhseY(|T3uul3%cS0Kj zYhT&i{YrZY(m+<{BY-+lt#w_FuY4yITeJ>m7k%*&zg4IyX!$r8>c1gBx`AP+#aE%0 z0_ZW;>+u>Xh7c#X$1q zXL&=Gyj|p*){Z>5!LSW)#e?r358&iITHfZo^#-b*qombW^X|*4X>iPs1NfHM(`N?U z&zfkyA#E>R(-D-UJ|cAT)5E#)EK+Jaebu0`D8d4f+9i)Vv|Hu8L1)b~C<*#HIYG!K zSs%ZhCuJ5rHgt;C68ql5+N7Jfw%M`1P)9^HRtI4CkHX8qCG>s_qWo1h)|9ncJMx|t zFRJ()z-~fPA*p8ja(g4YK8a&~Y;`c9KuUs;j4_NYzk}FtV7a(V_jhn&V&QnrSg$Ju zlvZw0--8u=f=}F8(XwB(D&bs{r9rild&U4cG>Us%!k_J?9(gx(j7%>k3=o+ z73PVCL(1{C9nUR~4NB113lEI7ttVH?U3w6Xc29#hWMBfk_vj1pKEd7KIlzN4tyZxP z@GMzVd8lVdN$83#D4i_|+MinOYo-jMJy~rp02sK(qz#O}xV#vthyeie-jkWQtAHtg z>{(CjgAkuT@SHX;+vW4Wf&>2@QEFIo zeWMb`@a`1J!4b9OTcuJ5V^tSdrTv^T~rJCrL>Ycq~L3MPV=4WqZ?LQ)x?>k62?%!4ob~~WP`kF7k4Wszgjr_g%5%c)z>RD# z2jjzcyOHv`#EKG{K64;fsltps$1#dgqdsY#sWS z<{;km5a!zJM#=gHYK%Q_ zA*#*52PQe*_rdLT1hkg~WFFpKtg{%66#3Y7GQ8j4fcnr2=Dq~Dbar$X<^pI~|Bm$q z+A5{nM2%=>yXwhA?f^|R+#9_oo0U~hzq236Fw55+YuzZA;+ROiu6Gl_1l}gd30B(- z-ns}hix3McH?y<=tjYw{1Zfd<8j`^?x_L>vO#In~af!&5P;5m*48%&)C8%5c`R$Fw zX}lkq#9?__mqiQ@asg5S*{iMxDW~2jIk@3vyYZ-)>w0fg&0U_lK%C1=?+x~~S#O`h zv(KI_42NyoV{%n&nF9!ErE<^mP(9lW)yIyzvZ~OJq*+=n55A&b0aXZE0vM4!6U0Pf z&xHJzh4g1|!2QAFH-Cj7{%S%mgMl~`2vcjQMM(f<+f{QtBK+8&*B7m2d=Y)E3F>sa z1ZY|Fis~L}%&JfK@^H>d^-@oiw7n=qU3rbs(6)*Iu!|hM9{c?wZK5LTTWM!y z;}KpkE+gfw$n50Rx`I_86qopYetllcs4XRbv3F%ELLo9R$Okp~iY6W)oV@pjf!kJ8 z+?I8h$8U=eLR+&~fy_!3u+47UB(4vhNKo*o?Rkm^c8n=_i#Fai(!Yt!6A9QtW9p+{ z9*GRzg{H4vW1oPVPHQ%(;}XRelq>Tpe*jEKVuAaxCDI}x!fVZ{WA{k8{g~|uop!$!T zCcK_d0{dqey_aDB4!F(VAq9G~ZDBm$=_!`ww{W3Zp_M_qd}IOuNLfy_8?H5+?P{!p z6CW_}fN!=;e-z^5!07N@0~x9qB4RC#i}j$Na5HW%BqKXx@{+$uhO-+Q3V0EeQT!1_x<2d3dIp9spz09JEWpmN zWOqDKz$oYhdeYM6DC-0vb-qPqrmH~5tZW~O|952Ekc*P*7PcB;A`|+2g53_c66T5@ z>E*B^lD&MX(LleEL32(Xo2Z9d4#l^}!G%Vb6XVYpwd`qQDs6KrY>-3w(spyVqUw38 zG1$xQeT*BDB|=4lB~M(nWWE?No2(OYT0khV$!NdPT-|x=Ma~8|et`ePMBx)npyEP|G0a+uh7< z4Hro0s3-cBULBV2H!+{|hsQO`?}Lt=l-?}-T+Ak{Zot_O5iz7C?Ko<@+wuvJ4oa}f-El`D7pMPx=?JAkJuJz1B5#@?S7o3-U z0C#3Fxsv*c%xTc+>LmwS}9(p{VRR=Eze~(8zxX!mps54Ot}WnhD8Gj*vdfh zY|qo7Qqh;SwqdqhIss63ePl86A)zY{$8fga4(zZ3V+19Txi|JQ1*Xi0^N^Sa?)bwC zqv$nrn}#I9f{$El>4&nXn&sEv0_U(-zrMY19a~0qDnPGd?EQdW1rMeK8k$(EBG?JQV@L!39Z*2h?cs zKI0rAMnkObe%0;<(s;A{l`Ka_Dj3titzp&i#YnW1TRrEQqzIEG#pCXDMx0Au;63G! zm)^xRf%fvB7R~qYO+e|3|IHUE3ReVN+3A(Gkfgu~esdsQyv(bMuOsDJgezf42|&l~ ztyqWHX{qA?v+NHTH~t4>;;rXD=9(;NtJ0?5wS^`%pTgTotjzNPw6c=1Gz6><&qDPA zlPJoFTXYdiAv%Zipzt**J7K@(V=-=6T(~%tt5vp<-*fZ%*DF%4Tag~g*Ea50dFLCG z?od(474>p@Eet8wuJx{GgUVFid`R8Tjzb;a#A2co$^ZZpPlx7;&84j6fO} zr19s^ZmKW)88X?G@Vc`BVy4qpxz@PnN*sR>!s7MxrNtXCt}379+Gg~7PI284ur8Z( zHNfjBn({z$2U3&}Mj{@J6?c==_PqOn)RMdc?=|`aF|PvRgUOdTPyp2OCAXwyUf~)5 zn`{refvfDvtzB=oJ8o;g&`ZeHw86RXmRphy_8CeWTdn% zljoNysLKXmGm!w2yw>AJ?rF7vIpucr3@}j5MUPCq+~MzSJPmQPsSxZ*-kR^9C<#x_ zsPbnWTQ+KVURW#nX0*$R6a$WQ|1w23YVhIeexl$)Nk~$yDo%yTKo`m!_ZKCuK($js z2D07w1=bst_zX71^ya?FeUhWyaPf@zR6 zNus=G&(!ZTGNjU9hiReu1#qsrDM%|BbBWIBqgDY>)t32=H45>iUun_~=`!n>iv*4& z0Jpv_Fji(B2kdjDsOAeFfO1RVr9K^ZL-QcwVSX&vJ<%ngJpJbKlrGVra*6kHs*r_M z@?gU3WcfeExNVjpgW;DR$ED+kxf7g+9*3L&y|b`xLE`+Co4LN<41pRH{v@mcleQz2 zd*DbDt!2#!f~s8NjDY6TN3mA~0oXoVJgnH#pkY2cNsYsH$eT_q+(sxIM28-kc!7CL zIa4fXc)l9_@o61t=?c(pb12U}LEbIAhkN>Fjlz20#evTT3LLxG%(2j-MEY9}sO+`N z^o|}{{D>4u71ry?crF#=!+6+7ehq?!BeLw(Et(TIkPN*5vLUHgw*PUi)lD^k46ZZq z0Sg<~jiZ@oVH6~H`?`lKlNxQEWklM0hCLLdSiy0WkMGzjGIPmg+CJT>r4!*b)?nDU z@$$8cIiWsb^Bg8k%!UxTHskgFT9DqClLP{GXFf-6;M21T$Q#!N>0$RYY|!N!jiB;G zN$^N3ZN#FB$L5k$)H7HJm&y0uHaQeLh#GC>04lr7Zqr|1m@MO=y)L~mR`{$u)x0-( z_~g=AE4r&Sb#>fQzV$!aKd7`OsoL9ON5Z{7sB`&@uJz5ssM(PHJkRF!e84n*1T1s{ z{|~pvu(X*p=FHV5-I!^K55aEke5JS3t94H3&{k;j%ch*Iqe<0ieV<_dwPwE6Cqm4g zd+mKIY0Pn!>6TIuH$O0BY8J7_=N(NxFdWx4q1CfaS2PLmi6HF#$oLu&PsyHQKA0`+ zTK(@yC^~wI{>1HG{aGJr2f(6UhK{@gQbwluw0zn&F=my=<~9kWsrkX2NxQ-08wTR% zfkYPB>bb_oc;&{kxu>7PVr=0;r_e! z{tGjlz+E`BQ_5>+sL+^qpq|PTFXjAp8JW-OkI~jGY>Rp2#>g6ECm*ZZzjrQP(G_Ia zmgt`PtZ@Hju1{|))MOLKxF-Y@T2)${Tb4E3F_f7GlNox{8n@jI2vlB-NjC2n_7FGeSx_soggOrB&unnnOl?&-AREA;2YKJ+9tL~_HVz1olv0G@vF(nx`n94fzB z)A3`ti}nmMPpd;U1~M>$^xjywk3tFsLE*}8l}ac)kO^N%Pc{X3LC2IMY7u_ z(Bn@#QNqSv=JaABc^W`vSTTOtYz4r!&VmW?krL=R~Tw(w5e{SAS^E< zO}t?DL4Ybh F6n~Z;2{G=&%KCRB}+(Q6qw3uo|Wa*>{8Epe#{tv*X+LJD?baCs= zSFx)i8S#;KfjeWIk{ZY|kJ{uCG8bm7utk5V?VvqKR!xz(jn#(@a4Yq!rw*wS5-a!T z`acikAbGte0>73L%2nCR+pX75+FTx zpxGwEXt|YMsBN?4V|h3tcJbB-L__5E(lyl^P$6lVuLbNUD*8X;Y=!rhJ0 zClCWa;;cFkLHM$ULXmQfW}2n8KlCq+!b9@)UIeruj#W@twU9t{^5u~lD7&~X9^dl6 zpZM&ipdBnN*0_!q@lSo^e05xr5eR%q;8QGTie?Y3O8kV^UQdzUgxh29{1G*C zQYugVqN`p@HTXbZ+V#bB3?2OFhQaXO{>fxz!5d?*t5nRz?O&)RiANeUELB5+JlnLB zT(aX8Cz4C>L4j@chD5rgG2H4*&!5Q`i@0yx_o)fSsW+4cfZ`e}>Y>bIbv0wB5orqi zKxRT$svOk5c7MJf*1)c~UxM>{S^3=>Q)Mqb<{npQZ2tiu%(r<`AN$aWG3Pvxn&gSO z@bu2VRP*nj-YWi)Nk@6jTegqoaMcE5HDWD(~=#`@ih7#Y|09MszqFAI3_2fokVzu3z z+soRCT_E_%_Nt%>1Ehp}L^-Gkc&Ry-6SNMUkz!C=4tylMNi3ImV>Y^-mYzq}v+*R~ z+hgGjtnv7=%*864e}l?M5lHYF&QM4&u`NPPb0SZGEE=0~TQF*~GV>-b z#wbZhnxav%QSLlL6QJ^Gi7f{+N#IjG@s(=;yuFgz{fRu>BAJw6DUv2-76>B%Y_;*% z_vtPv?VQ})e_S4F;RukdCmuTrwtm@)Xm?=dmHj^559A#6rZY5gZuoNhRK)kl!aHbW zfee93c-MNa3qF8aa8WWn*mrZR4bG!-m>#lEK9MkWa2Y5TD~;VTQsYl_>7Q^)i8*hX ze(!?UUfAWE_B&lzc|&dI9@otcq=Cg!&G(N@vS2N~o-$H018slcWuWLTMut!`=rEhq z1oOw8*GyMDPcC>d`(m>Bokdgqv^#6yhtIS=gF#R$6g@+G%(aM<95+Yg*$f(X1{iK! zi9GP*NZu!B;JPQGUu^1c|EuqNd!5`%SbKixNfjcTjVFV$%Shbuh*w zamZ$g*oL)?tmo+F@;nIvT&6>_5) zS@P%~_(K=3M~};Y?9;n)@gE^);VwKv{okCye`$MKSKQ5db9wueMzYXu!^e_O`q-3E8MTlJ1>p$@M#S{+@70a^vxefCgze4y-MEk zADb}R5+Oq|4ejE09lTt&K`Kwv6ACIsl|`lIofn1#P`O_MYvAnYjq*Eu29+bo_XZCA zJLj%1ag-2&tqIVhPQZN9t9=lbd}V&n!tvfG%5VPrUgNH+SUjMrDaoH7s7F4{H4V>l z2eRruq{`8@2U0gY{B~btY*of%K_G$^a;G(ZV%BX@+lO3eEUbSuy4X}^%`n(%%&JRe zsyRwNQdz?vPv%E6lIZ0}!tZlYtPpl8B~VcCrdj_=h{teP-naBgg2>NZ96h-P#((1Nv$vI3hnD;zec+wDvVSDp9iaS>^s zURR%pv8bE=GrdNpGHS-DVy#I>6cq-qJEX{LR**Z=}s6&6i z8bDM%ZuM1ewvJf2I5Mh=FG@AAt#aQZO~o1rE$L9V02EQ*WaLBDF{xrUH*8rs zegH(fyC%*Q9XlMfax}nVHeF6@?$1#ttOaVHGdR!3f>2_`jjaeQC4-tUe zqDs+=1PnPUpS6O%&7uQY^~M2%0vedEUC=~-`y**w-s7m-zqIY&K3#Znm?t%>_}@J0 z|Ci-aCyHw7P*5CstbF(OQ^o%WbE^N31tkAX)ITh{P*x1^xer|FZx7K@2ZMhq*{lf+!g)y(=R2PIva(-*b;1d_XL^7QW+nt##Lw?It_Y8cbfCesM?Yaeo66O`XTxxNe<&^7$RvB^(2e~I$Y9Nmoq5^1+r7#xxG z6(aIpAwOn@=^cFjr^%EwdP(^$dAaQ@@UwQI>x(k82R5Ueh7c&0tJp7xn_3OV*yg14<;U_*M z#rJ=2_=?vlsnBL09{(*A@)zOwYbnxC7Q#`S=BQ)Cxdl!jJ^C*K^6T;b+cbG&DJTi0 z53l^k5B&Z4nsdP78PmIkzx&D9{(~tosRU4vbT6^|A7=ThHyP3ci&y+cy6x`YhxMP% z=I!O>0}2X?g9GaKPJ#dASy{q(BpW^(qg&=b|L*jwzEED?mzr4(=`X2Ry-&bYg8O`g z2hJRYkrE%@A?hmne!utObY1{2uVBC{R^E*imv8Sy$HvShpYzG^aZfhx&MeVD;OBq^Wxq zKt&Qd6{P&zLk$2(@~D4r1A!^_U8Z5F@pyMu_2qAGKT8dn{WqlL9~&=pKD`m9(rj}2 zw}+bEh(Y}JAdk8pqdYvP_=?}Xu77xvfENK>!<>zR;-E_%MZD{XFMwa$((iSg?aSnD;@ zx3#&$9p&HorC9E&U_t~?j zbhO@Gh_hwE5bM`sZ|%L;lHM6{CvPkai-yZiZpFXI%biT|z=n(VCa;U2V0y!^7mFJN zE#v)K^F;BI;M$GWTeHc7p$sGpd3J83(5D|xf2>R230^j{b&9W_;Xs?Oy+=eGio zA!>)`b94zo4WFWV9}I`KRt&1~# zH6GcGkjmka5U<%i2az9*fk@XEbfn(VrwFfco18F&ZceqqKZsb!|rNmj4cUA zihJaPZdLO4;tiLYS?-*FmjIsbaXZIsFyXiHOfoj4+e-L8+M~g5WqmLxbQ5W>mAa~L zTgc`^CkvOEo(WJH_vvKX-@IcB1tRSOeL_BiabadiP%CRCBab}5 zs$(1T=n)30aYo0;36%91>ij&p_4@-(rU?0+hto@2AvIbeds%UY23H-3dh-enW`LNN zc4RL$xAFKJ8rBi93P?e;IvbDY1j%Xl*aNmbK6iLa5^8&(n~LU`4aQiY z*Yx((H(Pnd8+M~sr33(aUw8UFNeob2JZ27>*iRvLgIJQ|%(kbCwXaBiKFGCdGYu7d z!s)-?GTYy9pu94xpuptPD`mx5a*$ElcPBI)wz*w02W>j{;rQTSK8%XQE$7{w&H?cF zcX@lm0Eu)OpKl1WN@eF%xekBI>D25bf^i)@g9N&-&yU&?L80zD1OHU|sm+i|es_3G z(vZzlwi;uP%gc-m$LU~hFU7MRJJiL2(B2Z7r!T^ynWtOYS$ba4&V*VF*=}gIx+jhKK7Nv z?xk+?xP3D5lm>l~v2_Zzy7OwodnE?>yt9)q5;m--dNdvD7~?W5lL@X^krr{C^AE8F zPS3|Sv+EA`zV9%|*s+7*8^lf@&Chjj$y?0fSMk~V!IR!TM+k>%p6s}rdoPV~f6dPh zzD+CQz5ZqL79;_VSQjyA6#NhZ8?+rB_S3JEue0{ZY=oH38lcltgaNWuysLovp$e4R zHsG&<^6WP;#m2IG>l2gu7wri3QxX07I>p7UIMGz!g_Ta(SVPl!UHi()3lNiuCc1;O zN($%nYqNjTKm!Cttg=yVMRsF|x%7b2XXs&y^QvCy)}#vSY@UZ+@%;MmZL1S#B78CM z(UWaf0|zFhH}xDRgzK%7op!cc-Xi2>Zf#<7jFSxTC3{7s9ef5Xvve3&P_qBN-aRPn z0zu9lCpn^*1MZ)lROt-`x>jcQ#1BJ1&OyqWoZ(%C*WI* zaqqR%*$plkfqoc)(DE4=6tkDrakN#kQ+AERL}5&3XUp1f%VJFEluah+8{;h##HRj^{)goFdt zj{GqRJ#Y%WX*GtFi-w}N<~~P+q@apxH$%4C6jFuLHdB6c7Obq+`hh+-&sDd>Zjpw2 z@mvB98hbKY6YzyR+tz(D*GDYt#RuWO6k<}dX@!$>`aYDr*E5N1~8$|;nJ75{eEPuxD^{N+5uA? z%}jEr`EFT?sgk*mNQGT>`QosP2S+q#1am!xJWBfAX|Pzo2=rOa6>=8#ajU42+aRh2 zk~{d**bk2trz9OCF9CO~1?m%Q+tQ$|3PpHYfrF6nHCrFoLfw6gv0k%yQ?@l%7%&x4 z#4QC@ed~VHw}2UTjoW9bfAh_?-W*W>V9mTZaMLI!fSrQe;bQrX5KX(h95~upAtY_z zbz0Z_d&?vWmwK%Zz~gIul?A>9B0(Ujk>9f4coow(XcCPkQr~voBWTEN+%FFNa2=k`%;)l)A$I^z?RIuAud*_ZjaD@|9PDN)s-#&`)jv3o#)^ytsKQ{$ee^31E;PY z5({3$?0#FZ_MXi&WV*Gvw|aNq8*JT9<&(mlQPbVJKrs+#@-h>*u@FgW!u^WlRU6q z&|410jdGy+ACtDGgxXnwov^bavfRGQ5i4`XJns@Llcgk3c8d>XDszwkNdV^}3G~@I zt^w`U#`78AP;2?V2(D)pu3xr8yCJz&mdb-dU6^HXt6Ljz$^1q+jE||^Krkh6Gm!cV z4M?Ut2YzcCHTaRF5Js-uA8prKHT>%y&FfcB0=C~l(GNF+{>|6Fsiz|>*7H=PV;>l~ zcAhWQB70XhQQ%>`_m2;4Eg{zmo|5#pe{gxAPalf+b_s?Htrctp^KOmNG2{{L5mU)J zs0n*|!@s*UxXpNf)ubnjS>x8_SUt<_L>FN5(N4F z{`*PDggt)^diGUyyR}8Ni}+;XZ^9&si^nSr1Ifp)?OqR;UfrsjG;4hyF5Iwt6YIU$ zQ}g{z;0jp*tyOKj(NS@ceD&JFRd2xAf=IuiCtp{``)g)iSX)4DxjSm?Wgm*=fb!lH z9nSbm8Ps!o&xUR5Tf{A1BOko+v{noe1NDf&aY9V5SED1Y5mjVj!BBg9Cl9MxgWB6E z_uT@?=Yt74<1PFB(y;N=y4|usk=bOIGi?93+DAb%92Vs(tw}Ah26h};8W^bM&5QTn zzYXU0=2e%SmlU7n`dy1BkxEc!0(*3ZfXN9tGGokZ2_yuZm7nZ7E4!7enX$_j<(i+H zlPb-sJ3%J`Fx@sso+q}33d5N`KZqdo#Vc&>zkHJC9HCBW9o&=u_T;iTFKX1Wfpa<5wJuyCyIF zrZrD<^uJ&EKQGyT7xsUY;s1)yf5qqjKc|mtULy>#KKLL_elUi1LEGa7|1ZEHMI$P+ z$#e(vf<7i{suPNwX0r{$9KqZS23Q~c`C_#nUZ&V>eXgHts4LeY49|Z;^!!g3B~a)p z@a>_rX8J!r{~}X>u*lMdO5UG;`BvQ#mKuM|r`C%*a)<=Mu7e;=RI{j0ZuF?F=;|Y1aZA|jd$Ck|t zJg)V4H1^TYg_@Z^y%92rt`GhB<40Y!fqk)RS^s&6E>J)n*i{y`+9S-r7~N@LbO{rf zU#y(|1K{n8<`&sMUz)ygK;{f&6v}_`?R-bCID)Cl41OhZ9}k0Cv_F5hq6ro7a(;>* zH-EmTOk0V-uKKG*_HbJQwtg51%Q3Kos^j_TZ)ZO|TxIhmR<)n6a@6hg??(vNiq}JR zEnJ5K|4inx^bS$f~6;gQqojPa}EsdEzF#wn0>5 zsb?^D^6bm0B=5jf^_D#wLRd0 zh&72M^*;~XZs55V<44Q7+jK`{>b+>pP5*1D5JE!D0f}^IHj9}Z7xa&q&t7CE6FnXR zz_Xk~-S&>>FHHjjg}SwsPKFeiSa|cYbe`YRe1M}VY`Qh&hy3c zl+)HJy4GX>6zNqUu72|0IOl=RFfJ%}$4%}!K*0xBrL`Kz$D`}>pNVE>wPEp2{)ZKaio#Fdc^p*E@7^-fG_BBl0)LU#8|=9k=e@6 zuKQmuBD^=g6vH-OEWu~PhjV@wesI+TWCJ8|j``<@OJFxmqW(ncaR&fkIHjc~{;ZVR zt8yCfqEE{#)mZpSER`<|ra1Vg93eF>v;Rpd>?5&kSg}faWrWZg4-GvMs8hg5Lv6VyOMr@&s(Eh0XX& zLB4lU5S_xQU@j-c>BRd6f!YF&V41bID#CCH%LI#&$soZ+@C}=T&v#%eDL>hQzU#pD z^n7&Z`RU2d6hCl;℧9p3g-Zu0AiqU#|hgSAisT(DQ>pnG+C(&y@%gK1&y&@x1MBt_2kPFh>$oVztU{t)^W6qbZ=DyR={s_@R=!K4ivrLYoa5zed+&f zO6IEilB^Oo9Y=O5s9zN}`|z4A6u=%@A_4r~0L;Z=)3y;_pXcF7v<@VXSsAh5c19{z znxi6G*sW4E(b;v@^;iZ%O3LgT4!d*LV(YDv>^Du@_JG*y6#^iUHBNnv$AoN0^sFLO zgzD<%Nb7?dtr@{Y+>D$JJ%GS@;2<)aGA0p;i=w?Z%nGW;^-B#q+g^$y_f&OmHXcJ-fOSD*6;UQYbR<-VoyKfq`z-f9r3T9X%h3?> zCX`4CN-7)QhW%i(G0{y?V*A0Gp6qcc>@JtrVpLW7^icRTB@3MA@}fuQ$3iyt5lQYY zCGbY->S&^>p=6d#-SHPEv%J?<-b=?-&?$_e*J1dGGd4ACi`wjD6Jz`2^4y_N(Q4nT zf@_zttE#gDf7EEIbhb?8Bn&RyQ|hYHx)I=#|m?Y15%ww&r9WJVg&k@|jRLp7Ho&T1qMb zzhaS_ediI6)g4PM$WmL2FFhhmj3BiJP_sxT{4PwLf60&7XrLoloN(8~!FDpeuo)&T zt#k7OPcbG*JJn8b-{z>3W!t?Nutp2q6l3f3OQ@)@zvPf=Y zVdCf6!75Q=Eiv97ZE{u6ulNXTd8!?4*1c;3P)(r~kY$IF?pGmv{Hmsb51DUnkgRyj zf0c1mt>g=0V+NtBGa2OCG6Fv2$wMB~{8D&J;4mpiW9_b5$xpa$nf;SQTFG1Lt8hVv z!$bLR1F1dXDLThul6~pkZ@6~pS3ILpuL=$vpf1g{Ce&JD5Bt@^L;+F#^EqY70Ld@ za<9?VoU$t{h#s|A5c-|^RhzZz5o%TombgBE>W~{iCWcwm9j*D?JsBxps6D1DztLvw z+)}NVm)}EGs^)BjhRo<6K!v*MF>uRWGM4@$Bd>J<+mn1ygwl)~xEJ;ro#<>5o>2bg z(-0u+GduYe`7xh*^zSBGrUa>o0fIy_?Z~ubm<66NJwf(fl8LMKCR*v`_t09$*{DrT zo)1&j#W)pbA6u8E|3u5ix=HT89mEG(%9`X3!v-x*y$c#?4=OBrGqi4C)H`dyc&zV` zTvIaKWM=c_sc$!eBv+x}ZSGT7U`6Mvu)u2i7vT;*NaizB*wuh=kK*Nr|gH>D%y{T_jfy?~?3lW`Zq{8ZAvZg(X zQJ=^vpU~e4ki$>SEF!J%vDdDB<7c-|({&X@n7C;54Y@AoCt50a{1iF!3TFf zdX#8(8_*K$chnlB4Ke0Jeyb-I_6`wq>uPLN^{-jPqp}74DhI1W*l*B$MFe@6aodnz zviDjIUtW+Br|Nv=3bDfSN?a;a(GxH&LAj%95Fo#(n!6-xu;wI4KYrtvDn zrn1wbjVKIF(aP$(Ur|CXD+2b+g`ZJ3p#v-W(aBV>z;*{S~ zre*BDj?0ffMGUQurDTFyiuI9PD{`cIo`1dc1kb%|PoEhV?Zo7O85$aiEZkV*6`c z_x&#pk!o=#-E$<~(3gA?xs`0G7D1`45@aQw-+Ek&Ov)G^;WDSjl0BhcWr)3Vbs~bb zqh`zE+3lk4De)eyHz|OURi*zL?Q=pmCnLLDI=f=9`?bZ!hXoyMt*KMC5ufY(mP2XD zeIap@1*v1DXgu*gVCZH_aA9i9X<%lxN?z#Z$(UN7;$yyzy zX~Xb@&>VclK&IdNliJYXC#xUU9Gz-A%s@vc#cY9L5Ef`Axev9(>Qz9YyFeM&q=E_Rg_A6pbf@~MZMr$C5ci-(RZ7t%?7D4HAiUNwBa;5| zB9JQ(Jfl%j6Gr}lpjpwmk^6hi+zJrR`GaM9y!Qte0?T*-mjcRMt_3iguA5XiVo`}ciSP+X4_wS8y>!gmN@0m z!mQidFU<_wT*g{r?u{?LGa67iGyEpEThv|mQV-pyPt-eMVDfzxRdB|23^)*I5k(|r z9iDhbRbu}M5^VBi{VI@B=zdiuOpm2EV>O~sv9vS`T*$BfB0GL9d#8VwPOD#Sk_M%p z8#khtkn3D+iSBj^(_6;$k zm;0RkN-vB9(&d-qN&DS!!;xk~_rjX81<)5L^PYYE=B+~Q3@5eK`(iv6eg5Yu;rxZ8`_-cm5#sJyW6wgZ8XlG|0dT}F$t#6P>384h z>GVq#e}`_}=neFppPOs85_mgd){`mWUO^t69RAQCnMt`e_SRYE6@RaY4=UXqPVHNM zl7GR!$hFVO?x{~#ziUWin8k>bgxAo`>iveNDDMLu&Me1S1@7T>Rrc|npLZP;444gk zW3eu>XzG(N5;`lPV!3yvsp}d)_~Q(EN5e<4mxF)>p0ffnPm#5 z1g>3a5g^6giJpGE=jhdHTn3K?Sjfd1u#h~a4$NUx2nG*}l6G zLK(s_Gb(&w>D6?!TPxb+OHYynM4x@z^3XA=aXvQg}EYcavzd^ahX3cr*R zA#0*kzB05>{8InG2IWSupZ9ebR_>%l@we9*HX!$uEM4E=qCOv$7U~bAH4d$mc}hf- zj3TjinG-|f>idf^iKIIOT(!>ppLzTla934&6S<>jM|E{vk`V#7cK?^;lkU3rGqD>v zKu_EvGi-qviD^H5M*U2!vu(&%Z)mzyAe!^mG?TZ%}!Cj`a+VbKSAPX6Stt*W-`6swRAr zd5d>E+M;cIv&(%dW6#_ih?O}79h+oI4s4?61vj>t0h_k2xgvK8-AIrj>J0{Ti$nJ78u8@KVDaa2 z=dV_XtpSUzn+s9xHS;GiBj%yz`%$MN2_e5sm(rGR9A1T_ou3z&o5s$3=2UVqAtxRh znR-{7BM~XVqiR$^p6vVxiSVtfCVtM$`)at4;+EUK<4nNHY`qHz@0&UU2w7JJzxfq* zrZaV}5|rB{aaZg}`Hd9U>X0p9vsGPYh@M;ZuC^sshf6i-^Z683(jW1%U-XVV*|Rrl z#wvViUezibu}m>4*M4K(Iz0v*qaD7`pB~^h091MaKf$KA=14_=b7ZE&ZN}K8Jy&TJ zbA-CMAnUp%u++`_5fhN;%OBmkPbAsTRP4j7b@e_j1!vOG!KBWy3aQDqxWAy{LO}p7 zTD5n6xQSPlVWN3~eM?5Q?1BNi30hLxVjES*ZOT_inFy<1v^*_o!(#^FYY+*49vYWZ zF}>W{GzifgtN9GnxW55#*9lzb!jrpX*i#K49m7S$a0Kx+X}X7p0^2^lS4!v56f$GS z{smqsJFg`7RyXU*=}*^UjTk1qV}Bk&ErBc1ApkCEilD0cNd~SD?;ozDbBX?v1KM-d zBxdi0%B?P2Mo75NOeyD-4XwrBT00mC*;xTGs z<69@86TXlJ0(XzTyJnj9*!@P&@X2Yt{h&|n%B}R-pu{ZS$52dicJXplkg_06uvnu- z02{MPuBJU(eR!Neo30RLLMtAN*F`=vb3t-mQBCKycr)<46{?;S?6^QV1H$3i0 zq8q}gTYfs8R81`}QadyH4LCd3o@5*PlI*EzMibVp_DNbwGSJm2jBYe3dwiP1aAPr; zFn0bAzRJ{IsY?}Ml4Xi^i7r#Fi$JX%V1Dh?knY1dB}#M7;;kpgLsdX(#nBtn$V}L~ z_tPVd14{&Jo0KS)0q=z(ysKk*Y9sE~1Uek%#}J>St%{e957}^CwbJ86$Qd71k33|e zKul2A;OMnSL0=S4wQiBQfU#J`%jYeRWtM`fMS=N#tjDqfrW(W~*p-1wfS);@QLjhb zkLVCnF+ZD&4{E&4_gAjsZ=BR;C!VlZ{8Z`9huT5Bp7 zNSbnja5wlsz^%)psH6kKpJNx;1C?8z7u`?G&g;(!t}8^tLqmUJW^ zND((`UxZG@fk5b8dYhsObzRD%`6JvY8!ulK?lE_X7_f9JvYz}JAgHsgtCp9rAhv>1 zAh0)FA^8Xh{i~kG_$^vN$@|3{4KGx*+^t8}7t;I%jAiik4>@HO0fhcYbSA(`07m04 z67zW`DTtW?QDy4YOH?R3qcG=;$LDDFYNocoMEVRqXrz&qxbP|nndk@px8Ty6^xKIlBYV@Y6_c{z)l+%+J$Lq_**|^65d{tEu zp+%P6nA<>MXNMeQgRy44zvsXvGtQYJ)Z&+J`CMxV}I%8Bob~>J|^Il#%0EU=X%aH5i0I zv-}1tPL=U5#&@3B9-%l`1*RQY2Z(xBz7k!5DMjOuD0XKiC27gM_M*wWSk$Wvs^KxZ z3M!GRu2VXrMpm9C=Du{9SZbS-c_6tGRA(d{#mVjej7f>OGZ-{~8CxUh(RQwSdUUBj zXn4t2-^X8ddFAL(P7vaaWf5mGU==V&=pHGx9_*klOWe9ayii;RV-N44$O#r{Y6sIO zzP;wkiR7SrCn{9d92}0Nf`Z39P>X@f19lVfZ`uug(o+4D>`VvFtZa7O^UR0*`wO4T z6{U1ec`^E_+>g0b{8alH1Gv(nq=g>O>LgEu2nu26&ZK`z5v8B%n!ZXQUJC7>!JH=t ztq-prkP@bx?_V_n;k9+}_Q2hFjf!w(nUyCiE5lKW(qH7t=OT>n)Y%S=#92BMA-kT} zSly1U9!ey|D_XA9dd>kAx|hScoj208R3&%(wW^rPS3n6}5ccZc(ZK9&2c*{y2$u`qTj zFZ-?&bjh7@7uQ~X6DFO3y#ct{F-h{D=>7R7|N@Qz!FmsjSFXHikId8wG zn!+(^4FEw97AZRZv_S3DJ6-IznbHPSZq3BxJ}1OLFsT-hQf;%|RHoV@u~0WDsNc4_ zX^tG^+vyNfz{HiEJ!ioA>7uB>LQ4B`3wS&4UB}Ctz_8btBsKp&%+Fd-Hv1I+>tCht zzM>5R25Y1}vV8&cm7y@a=-8o__!%rIOiHKXrQJC5z-v;(1~rz<2eVN>Jec`wu+Yd2 z3PYVes4N&M5I_c>GyuH%b7YzK-nYLMQ(n+sn-xu0C+s=;sJ5k+kDG~Awx^zH>|3Mo zgIX#>+gYY?-{Y%C$amUe>>ckfy%{nr$VlW#Kq(}2Pj4ba)%yVO=#vTvCz09M$|D?- z+^Ptf7f6fe*UKcJ!v{GEOP0Q!ei9yZO;7}p z>(l?PVkwyW$C*JuC$xKLzh# zYB+v+cbK&{OGpwt+U!^g?BHhn|*mCu;-chm`(WZ)7t@DmU6GE$cwdOad-y?hrWY7$l9jzUxR-e zYBeUY9^A|QU~om&j-y(vZAKgDg8KDc<-R)@m6g+KfA#|S%V2^}g1tf8$5)=l`a9KX znKj;DuSovom%k7EbUK(t5rbwL>wyZv!4h_9I0tXCgqN64j~3Em!H+Q)wKN+{Lv~_B z0&4>T+PzLJIi0+RVx55ciiQ&$(J|FgHMadli5TL1|BXU7`>CVyQ}=hbP6SV-Gf-aJ^lumINg(1%r^5$`QY(S&t!F*av59@fTmVM%?V7JH>&H7VpDw;Xu8#G`sd0e4;Wjv{wMpR^@+fea z3Qg1JSsOR_^39tE>;*j+RJI7ku{OuMmBeFAXokn!eGn8+T^a`2^FdwZIcjNxX|WeQ zuEH3ygDX&3V0|Hli8OiB(qeFMYGgr&xg;n>e^fkDc#a8ow-+R9B(7p*)tLVpQG%kKB&{h23E>L2&heVO~;ZlAw&E zJ~PDka7y<>^bWp9H7Efo~Nn~k(GiyoS=&iqF;XBLjIJh`^8?FCbbux=~cY&c$) zH$Utvmi+L=+T+_`Ee|#e)7J?|I)P#CVvMJexyotujFgS?!qh(&Oo%K~Y%Wa%Z_b2}jdEGyX{FHDC}fkrq*lNPy0JG`7oC>!cp7(rDi2C0OC zy~S^yc!(~a&&md~YFJcs3Xn7AsOe)L_2qUC^%rw-@q;UkLr1EoNzU?Rv2JF`gr zx!v!as$Ggm`*B$;`%k1=rmeu!K+m_{(ky*9=%l)T;%U zcGk3IeopaJX}m{Gg7vUAF%pae)P;NYb=qPeLrqxMumwr^JTwF7{H*!z%wC^s(~GJd z)DSTCY-Az)`u_59=207Yg1lL5Cv5SufxDOlTR1iO{_I4GfB(<7>rDKL5pY zbD#GHMM0>Ys)4Iv?L&Z4_+FrjW;o|H3h-F@d6y}DOYXc?EAEu7z{=Az9R)uKWcv3U zN3cE%?pD6x@Tvo~;CkBoVz}ZlT8ZL{Nk~GrC~XZlzp1w zC0ZlJYrB}GQTk$}e&cpT!6rUM4Fb0Ol715FXnqTuRfSrdn7eL6nSV2++L1`QSsX$tsQU0wAxsmtvtdluxu7IhE0GMr2M#<93r&$a9YkG@sh0+sQNvSvOMi6xm;5Cue%Yyo1Ay__W$~De`e^>pCYt7-^|zHe2g4b9_()J0_P#S3 zLiuVLxI`w1WS8H$77j`i*IbL$Jz#}hc2P_`@X;<_ZXmqSDRG*kcaoC!X^Q1)nJW#{ zA2>Z*k969r81MfgVVF+sPq`*XG`p;Z7T;&cjy{Fw6e=+0NhmbFxi>yrS>&g-dvUhi z_g+QV*U$5D>4aH6gNE%OHsz0WacXg-ynu3q7}-zEXWnle!rbm2+9C^G`=vK=8%37& z&OKet;RVX7=d-8MpCV!!JH+?rN1gbjxLRcY`5uSFZ(OSLWAbxvY));l^sinz1DMU4 zmx()SMvTnOldRl7J{&F*L0RS(_1s!m8MxoHKZ90f_)sqKkyhT555|v`7wV<5)Y1O7 zr|@O-i{bTBs=@1q->9yZ3H4Gr$^^&TNF((8S5aNOL!iH`<0X*$9(OX|*!e!I;_WImJ1|vMd4pnl(T34ySEq7$?%m=UXb)@~Y|8OSYL0$VwtHH(6 z*Vd@FnD|u-I6jM~<~fGYiEbx+DYL#onNz0?8oZuE8!ooLdQzWvK})>-3R-+Wahr%J ze&=rm=IjZD=C_BF-z(QP5T5bac0bE4P=fjwY7##1-0uF+(w}UWB0K3fLeoW7U)VkkZL^H%GHk*j%3yha;i>s8o$lAf#XpS8*s+Yt?@EK^O`0E zCCeAjYm>a*1Ls>K%=++B2-luG4j+hsqRI}mz_bK6OV5=Ek#3}Mr2-f z)q9>lpXF^8?jMZiS~WrN_BY$ z>l<+)T8Wc=RzPCzR;_f(-L1Oq>n5?N>GhYkKJF=e#sArtf&ka=#zIKe z+(7>L@q|JLcZ!x7ic*P5K4)WDY-`}0_LI{5< zTnl-&@z|G~7l*+n)XIp41}4Z_GxypnN3BivS~IiM!Yro#U{L3~Yqtqcp|aPxIii1e zj?y_=BMC+*)OG(+k*B24_5MatJ^#^Id(#N5g?fFI{|NbDqavAwW^>zg4K*w@B zID(Xs2y5e8wXxl$!OecQ@kmYmk#8oizMu~ioL`G^ANgafzk3Qu>ikz-p^E*hxWw-c z1D}5=VKfeU)bLFuZqK_K)>yU`*Iqj0_T zVc^`pk2N+OxI%)|Dg64jd9!E|_IHo?V=;a##$RInA3NiRoBH9VSRG71cE*pL@ndKF z*cqcgqN)ExX!}x=rx-N&(X|GeRuiS+cd56kRHCpmCAd3w_se;Wx`$<5%WbcP3qTxu z3B!=(+WNWq{BM^M3U518bU?n@nm@HYq0rgOTSNk1DkhG&ffnbtN~~+h+jXTO1!RO8 z#(6*avx=G@Nn8wuU+~|>Lg@HCnK;E}QCdlVC+hkbhzDMs?egFSwFr-wpzu(X-K&gHjD*$w$iXL(8CBMK_X z9c99Vj;~oblZHGMfH?06)mtU~J>$^P9XY^(%%C>v$psdYykd?lg z68JqSsAS~%inu>}zBI@X%>Fq{NN)Z0mf_%SQ3qbX1sL?H(0g~;IDUEi^2^CDC*yjB zgnkKSYryqN#NFI|X7vfrJ>T8?uQ6Vq5!-R?LG^m)!yu!36 zfNmNnZHpbnSE%-_w@Vfw8hOSn2KV;%=-c&gaWkxO%(l-&RXa;ZwCm%IAr`5yn!S(Z zj#B%KwU@%%^Wr1WxyXrw85k#NOFY^RXgc*iG;NmEO|WeR5P?dY{c^ec=qkAcPse(F zeCXL)mLVxA7?10twJzuRUbVuFpuN@(Vw4xY?R56fyTj(UbuA6a=@9S1sfqXav}-mF zs#xh!b4D6c8!S6u^_qN_eZtcZw!HJjM}kbF&<(*0ZFyXoTBvtW;a4y|GzxOwc2Qbe zDpV%oLO~_s&>T*hWy<;U*Au~0pe5qaNRC!5Kl~~T^eHCD_TchD3mCPWSKNBthr3@> zCq8<(`q?-oxo=&!MRK|)dHkx4_jtK$CC)){Ab5=rg+ko3o%Pa|dfZmKdueK`Xbi>? zdF8D@NVt$ykx(vI61`N%1lCjAj^!pd#`ygS~5txMcpb#j5{ zO$EJYtbqGNO5fVzvS`LlOwxMS;ib#U;h75O{3;m_Krq@GdTTqATo%CB8*E2Q)!7#OFSkhno}DkrA@i z5Dh}z!Wr#_sScMmm`_X{wHUoxm{=ceHlPyQZMjMJ1-lPwh8(?&IYFXktpL5Q15+gF zfPm3P3tT~QC)Ct7w%xJzko zK7x_iwpl)AjU?C=Ge?G3pbh2MOfW=}gFc#&C!Dh1rNOBkd4vYb<@tF+cbKk=6^xul z;Ov=6k_{R&99dZIoV-sirv>WKkhwl(xiT-V1x7g`t@2vbGsheh{Z-IVR9|U?Dc<$$ zssK@%Oj3pdm*|?km48@@4O_F(Vw{PeeW3KCR86q{`a- zcKewMC2pUd%%Z;YYKs=}Szlaku@(sIJOia0RY){q*XEgfMIG+gW7FU1c(iWgF1kx3 zQ(FyxNaHcT_J!M-!msQv?XkN+IDN9=$@swx`!l@SHZtu}hPuSJsJU%Jjb9LQmIPtu z7Tx=<>B=U+OZWBY;#C~oCtCwgBVfS!W5hh64w_=<|VDe*1px% zwh8b$)r!bP9Dhz4MvfAl^Y=_DAazh+x$KKvQ#mVp>kY*TD)Q* zcKnkl0krUYE`fqumd?M^Y#0L_?B=0OU7ltsUsObs-G{D|dq}IG6(hx`nb)lE_BSWbg`8qBZ@Cb|anYC6Eh*-p~jYONa)wZtl9aXJ!ZDE6k^; zZfTFUr#R8S7v|$rxB3>XKy(tsqA0>8f-Tb2drNbZCe+98G0+F0jrfmj)X@msW<`A` zw(STErX_ifqT7wkY}$0)1h1BK*_pke&g476I`UpfaEY52pYB`AQzC%=&~2u@?u%Z29x^_895LQ^ffD?Veh%r%*4UK6UE+?@_7qw^xBPv71zM zfM8p1(}4A&C*|ECN*`_`t`Ra9w7Ukx-BZpwN?v5W=u+l42S9^%4E`6<%=j0e)5UyD zPwm3e1*_qRtpC0;`glLKIlFJ=ycy}JTo`v=i?pF0STV1NiG#XA{o%d7!!j<`>fFVk zCv4I`pXPt~jh7Ze-`J?j(8!s`a)mWxvR|yy$U;PNkTY!AO?KSs75OF4gj)b?c`UXn zPp0wJjTxJ^Jl_m+R60~54aSQ>;S@#|#k4)&$t)W$@%5Z2vUfn5<&qEPY@a3!PaCl2 z4R4O_G#*g7>gnbjaW95^0cm$AyW#!P6kE*PEp4et;;nI*k6PXOQ*;75-i!%q=I(aBU4ivX-BrWh1&yVJ+xNIR8~p93I=%znm)+tANkm42L!wSma+5>iEr;=d zCG{=EKkq}$9%~K4?t>Xu#!aq2=xUbMs=sptRx@GpRh{rXHY3QqvsVuGle$CtJ#-7} zayXO&-7EE>61US}Y%@@t!p2=(l&G&!mTU00pZeXFOVx*$oE@2_dwjBNGt*_RY+q{O zSY!RaaB6+uA!2r^&@#04S5O5ete+gi`mp~wA52fb>ka<78pHqTEd0k)Gtpqy$cG|` zdFcmD2Kt|8F0fVt{S;gZR@OVJ8zm>|#CO<7dRKHV@&W2^o9GY2|A&+Nf49>Q!~b7| zYyU9OABO+`E$#f_@qc)Hgxo*2(+`jTBh&!_#E($te`lz3x*6o~Yprn@Z_9ZuN~$Y= z3AG>I8Wl8ORak4+h>Y#e?ZUQ#ggHJN7ifu8Wo#6VEy5~!&2U#o7L>IYpzRHpm!>YA z%o%e)8s+4wXN(!2Uj-R+V52&3E-ID$Z8aY4?=vau2*K8U1K3tTkN2SxRI>g&fNuKf zXvam+vLiO1$XbLLoFKs*o-Sl04@%+5A>_8`ZY*e!o2!;{AU-RvG%!XLYu->$(w-5i zI$OEEIN0Z-#la)~E^?`KFe1w`ZQdV+Ud^AdeFIy5{ha72-Izf%PHp!Td$kV0q6)jz z2xEMU*L;8$K^g1KTYzGf>+8wDIC?QxyC?sm0QjyMt<>TFw!u=#%3wF0=E`w|EgPd?R6~ zXfBwI6AOGT#O~GD(~C(~G7*Fi@lQY=2V%>AC;tH$HGR#g0=Y3CSMkQB5N=QV62kA_ z64s%ENru_2(SsO9YqPFOyWq#tAU+I-?!~N`%fuk^gjZ9Nlv7kh0m%9PCpdLd`VK@X z{To7yP)3X3!=a8|nkNyZ8IP*$Uyiv)9OyyTjCU@Ifi%;!b9Iw6vy#BJ6y zG=lGdz7>6HhuH-aKRI-32>}2E zb;If$CFaFk?XQ(?#FDHZ=Vn2}Ben6LfZFk4a>^!&nzH!m z_Z+=MD@Fl%_Ci#aBIr4&$J|h&RK`ke2qDC$=L}?HWq4$xA5KZ>FRMe7*EyQ5f{I#$ zC7J4)iHw}+(IkjmW6;1xena+57DToq5`ZDU)iX`fNw)j(5}&pR9t<$gtmyh=m$ony z_Up+ed|8@$^Nb46?P<9?i?GOjbwO5ybzaVtU=+C6#M$XkstqCUeI*WhYiu*@QJ3S_ zDK@GM(OlG}NIO>uk?S>B?b*7%O2x!w;&c?}9}g4mG>E4JdQkC%4fZLMrlGgGN%0*t zj8Pgs+26F3EGpO0EdiR4kV~ChT@gdHqtInevvwvv$!qqn>@{8m7(^rVTSX3ZV^;`X z4L^A3OrcCfOD^a+InbZ~et~s~)K9^f?K^ye;YDho)-(VKZz;=nX!(>s(%~kWvi!<| zdU+lJmD%Vm zM>rGjp{h>bb9Ov*?3)$l)$b5gV+Hx%_9)EuC`iZnMX*Jm6YDnFqH4#PHXinn?ACRg z;03OUjtwJZ>?x8h_Z!(ocH0;=fo5dMX@hg)YDP}EI?{8wH~;3xd-;^9#t>B*e$Tci zHi25aq}imamC;+-n6Y$RWas9gkmX0mqACu+X_Qn zJ(PGRzz!RK#ZJ(fAr(&ewLAnXWMq8uPnM#781!j(RU|*qjq9n#;8a%z82b1b+bVE+ zpBMpKb74ZJ!(oZ<-}N*5rQTN}PCdtMcFEvnC>TDEheX9f;-c%;v#r|8`yr>xa=uNv~$n`&+|G06Jt{VWDt9>Q3aybNw>@l#~n z%WG{Xnj8Gbk4Y<^jt}I|dv!Iz0I4INmX36!FxD@xO^xYjk~W&;y1QN0IvQIGd#0Wm zn%EZ1=hh0qrTf+*7?H>6zfl##n~g|O@Ofan-)cgk#;Q22K(=ak*eV4D63_^wUx z+S3Cm11kI8uIyXpDI2&%26wYrn7+UHv3W4NLV`(gbYz-9H`Zxb6-*p^S#JBk!aHtu zQ0kr83ZkR)T~4f{qo0DcKzL_#!p&JbF2FM<1L|2w@UVkmF5tR|ZTTjobQLU0PrQG9 z653rhr(b>#rI!wo$w?!($?LXfj4K$l>QNPuI>}4=WzSH0uapaNUXj~iV{4Aw#bc&T z#jLlxkX&%Jp*eI313S2gbjpeq@Vgl(sk>`|&&~Pa6%*}?gwP|OAE6GP?GSzijW|l< zhveyY%5Kr_x%@%prG3Y$BBv!GQmjj$O{&Msf;}5*gcK8OmTGWL) z4L8%?>m3D4bfO8I-P=V7z>P_p$K9Ij+Acg?g|y?#scK%Dy8CSPu(lMJy!8q!@~55e zGoW`LSD{~yoemdDMDEzRN`(GF8U7vqp^}a26EgwWhf|@ad6AE}+hcb=PpxISPad$ivj z?eBc|x1ail^kEJrE5oENo_SktvK6lJfaqcRde4HLZN$;fk?6x`4P%<~UjLl;3L5`> zg8;J>92-M~lu*aWPp=pUQlhxcctFZkj8 z|Ca&E5AXkfn)m;|%HuO7VMwR!SlNK8#MY~#SDP|(rCOsM8a-WzNag4SVX_xX|b_?XMVhLxjBS0roKlyDl zoyc!Ya2r0g(UMO#BSnX9r^K1VZo$R}*$LP}KVMT?_2e53fmOZkL4=vnUVn;{RSGBM z+gs_N4LW%=C$L0OC4a8k-)LnU^>FyX^$a;tn`+|!5WM~iVP7ok9EVi4uUd5z%Tt&PX?p?4OkssW#(jd)0 zAU}Kni5gbpot13ys@%GypbVjHYZxFS?RKX}9iHf7Hx$-wmyWpx;nc$mygCa*q7PP2 z+EYM#f%)@(*=T>S3Ge5yWmX&l+9@C8;&Ej})?s^af-sY6 zQK|+_7LT`%#973lmNL@_Q~ZTmQ7UU=^U!}%I*)BwAPc54z3>oUOk(++3#Ihs9>rwG zuRs)gDCPuK!~tpX;W~Tf*uQC`b5{U*Dxb@fx5tiPHc8{ec3!3>G0{3aR}!@pPT)U&Y*0ezDEjq)*$n z8!8;Vr1ep4LO*;u1r;z~=QVH+j$!JGQEUAFEO~~lAzz93+c;f5(v*X&ftq|}f(<~} z{LiWXKh-Dvoow{W(B9uUi1$%~G2CX(xnA?>wHPxuI(0e0OJV*`n7KqKkg;`nYChAR z6R7OFq@6m>js(XVnkzsSXYb>c9YoLS*l7`{wr z(~6ehV5@=hCdEl~jLwReX~0}Hfu~93XrJG(cl$ty+f>oa$E-d~D@I2Nx-ytgpW-J> zQGB$(?LkC>h^`8nN$RMSK>ro;aTB4jEQAA~AKX5Ho1h=F`Am8*sDGhD%RXQXDZC=p(B$PRO9DL8)fXV)bwrZ5uB>9Z(-5pv9$pS`|0@ zP*Ml9mDDetDJQ*QUSI@S{TAagc7Ot)R&^F-6Pv-8|Apt1m*#+k92;7H2TgjR-Sio# z8@-b^yGTpN$Na=)>1!Qf*fs0CtjcE%FtKA z6QolN2?{p={HsTDVoh>YYW?>9;W3b z4b#zgpDni}d~`Y+y@#x@2lsK^RGU4o!A~P?%x`1|byAwYS9bVd0&4g=?^I5U*L&Kp z_2Xjhg3o`e*~G2209ZmW?L9Dsk9CUxY~4k)?S%OR*R8xqAFQ1`?FCVKc{}hj;p^7O z1LdfrYHU-EhlGaSpBz{C_2?Hg`rP>q??Zp+cmIy>`hleVfRLSjK*)awnEn@H`yUYU z5AK9Tb<4=GgEVBB6#pO%|A%V5A8h##wmkdaW6OUA*RTNf44FR>Cz@6KDU{p| zgn6qAGj1u~QJ_Mv{iwdWE!d&ONW7&zWS zZqEs{Fm=wYe)pc44I})D_z9mbg}t-EftCmvM(F`43{)ixb}$|FZ>zlYBl_8>>CcuG zS;JhX3EMx3xsz@W7oic$Dnby@iI+StDxnj`4f(6Exy9EB$eOifXRTh6D@7)p?!uO< zW2<9i8jtAV@3-aIzjH>th03PG4!uP|&e$#;)6T1x8NIp0#|lM62Q&i^Q=x>1b=hR_ z!Be{d!p~C3Y};gOFPY*Ed&hrL8+#s@ z7d(efNR;c6{ztEf0t>{FOlwl!7+(yVSIk6*2v;kR{k_cf4rfE zN7iCUNKnkiY%N2wZz;oUQWrUjOLsutuadTRfUnLWwJ+fdKiaw9EJNKtT|jn4=Jg3J z@+v&CgpUrP8ss&6;=j@@6Lm(N6pcLDV6yf~7WN%prQZ_$Eg*>=$xV5Vipgs+jBc1t zIoT^Ew+^knVfMPtQ`a}OpT^OK@PFv@It-p{Np5!oH1;RJ0@4+jpJ z2zFuX4lg|P_Cczq?q|4iHtIQX8*|p*v|C#UV1IHh`5xVFqzkPS-vAMQ&#A!z z^&J1k9b^DcCzsRu+%Ps#s;`%gG07$k5{Y(=rsx2NwEadpT#)sf7>t7K`J1Um0k=O# z+xrA9W=ysU3exFP>}Zb+GP*=w%jRBT9O|(N0n)Q0@ni54Kh95!=sf7O&;4e%ZwFxH zgfN;{xaUKAh6`+jqwqjoH*rHykcL=mk<6K#bW(@kk<`|p!J>A(xQn{qZgW;$YFt|c zW|cSv@4g)`=O1})rt4d8D$0ytY`YXtI(Bh}k{oCxji?YjwD5)akEKoSmPRPcVTmT_?=*t<6toe$zz-G8Cgz1=)e&@ZBC{zg_P zaQm3f$-F;#ff72C@c~yYtB&~OFmloX@r0l!q2#)wE^Z7FBM-)>OZQ3hgaq$wJ@n<` zSLC$9WgT3k?GY5NTs;P#*n}?Wk?FE68=FqEmzXXI_bkhT(Mv6n%P9ROm#Z|a5Gq|`uRv{$LERFcH)`O&tgxO@w1mB+Dp4_ibTcb)M+jJw7 zPgLtmH-5bJIx>d7G%S-Cii&{{)8$Ei5RA364X(yZuCdEitcUDZM*Mx(3Y$^8^XtyY zQ@`yTSMYyv|BkRWd)nu3;TQ0A9(-@|9276h2&-^IL|o+^>>WP*;fC&aIP9*m38o3@ zpWd;_!McVzw|_~K*66;7WNb`e7A92!wex{MG0aytEfv)2s;4E;7t z2Y&C0;9bC39h!dt-1%3>8?0U5>AP%)d5U#kEM(foX>$m`oj>S|DQ^{eN&e|#{O7~t z3Qyv<-Z`bse(dwHT^I0I?(w}OI4G)}5x#W-29A%r$>#}MIL>|?99KA3xn=#1Hv8%N zZI9lJZJXV(s42tWe0r;_7&scm9uKlsimMp|#IFi1vsvsU2L3Ig+3LU9z8nYbGNPu3 zud}ugp{HW*)I`=oy=^~hMSy-1dc#u&tRF691KuqaX+O?-+MdROUz?1aHD>+lcuw$Z z%!NKD);j1)t$m<|LfBBwp7qbiH9^{1UaY)HpYxx?&A|UX2&?B=|2+OAm>2x>*3nJ+ zOcBI^Ufqjlc8&G7ot_0h=L$Hob(3&gh#3Inl#H4}tK`Z~X63h@B4dk!ln@L!iCIrG z5$^zQe-TTe4e{h8PnRu-kP|^4D46p{8~&6Y$^i2q=`UFAAP=glB)JyPbu*Be$zLe$_B?M$x)i>14JNR)x3~|MOR*j3~|~&o^zq8Sc-5RC|G`_XX3XlA3P) z-(1Kds;lSL*0LDUOg@_vKN0Bg&MJp?*;=I)?|u3p=)u*1W~C{*&G(oy<`h3p>_qMX`$hdg4Q0f*6J4FZDt=x<_vSiRZwcl*E(L zZ9pi;>__ie8@OMkXIgoOrCCG*9{O!vlJY4Fy4eoy)y0SVvA@a7EnInzB&gba^7W{n zO-b+zsz3+j@aKZkE(0x#nd?A%NhRU~X>Wkiec6S1zWxyu!FP|=tdpiSXEV}97Y!}d zwZ&k{qsiR6SejUcyfK~QA@L5?zi zl**#pR!NulvC-Ti77;jfPaW%yxstG6+elSD__Y+Kj{*T-ry~DFV)VQc=pUVfM6yzP1w&Q zKNONAYHgq&sRd|_?)2iPe^x5f+aFAs;$xtPGb>~y!3D2_YL@qS$sF=auWr2MQh6wk zzWN2;pW~wTm&xRZ+%{cJKgqD6Tf(Z8TOKuM4hDJc;j#MdZ1W6h8#V@WH40Dn+T#zQ zSbPNNb=rLh@dvE&Zo~GyBT76r0e{S3hOmEIynJ+6G^5xfU07PM_)1jn4br53G%0{o zv&@1CzJ{$@A+5Tj2YUxXrK@nYlYrN=Ayge|Vy%*H;@veppRZ>_OXHro=RV2G>EzSM zETQsr!}#L8N09m$qxA#~EQblY-90H8Nu<1a{K9m4Nep;UD zr_zY@xA9Nr6%FiX*M+rG^>M@?bj5)D%98&&;t^@kex5N1wl`R8k+Lg0&dc;eU5*5b zfY#X(<3HGyRl4cT=c{5?{ z1l$d*`--x^5BktIY2<WeL5Jy~iuA(Uq6BmcM^kOJ2PQfyiUz?t9?{KTcXpFLr_0t&Tv}{-;MJg__ zLOT!0&3D!cr-&~8>;!cqvMbi3x*_h-oYC=&v|INaezqF}MP~jttD){e=V>dsdleVr z0K`)pt?9ogZGTcL5X%v z4ZfU9%=U?voitw9g3fa&4VkhMO>8( zL{iBu{*J=0kWV{lGW9eG^2<2*C!2BMQc8QE0{%^jCJW^s?b>Y35@bxsk66ps#Dt(v zd=@9zv$-!}|p+RW<8 z1GOTd&ZPCeQKcHz8uRLD1GzXR@sQx=Tn6{ z;d^~#kr))j)35m$?Z07xN9p!}xyytcz0E zh?Xqut5VUlf-W0kY;Cza3G1Qux{j8a)sBbd#4MKiqKr)(pnareKoKJr){A&!Af;)B zLE@eKW0XAKJ3Za_P=jGcDAsqx>NDCi3m0Bd*-Yd7@ zr~E850)WP9!TEKdj|u5lc{mCLgDnQ+Yt7%ny65^p_MNM!#gJUK*jt{X( zN@;2WbdFNa2F%T4$QZ5K`2qB!8wr=fN;AP!#v12#@15-6%Rs>j@>y)pNS2Z{g& zBLGd~3G+Fq`2#!?3gA>Pd=l330nT!rIhFHhRk=A08KEB%3oZcug$)?O#K;)4>1>+gxrr$Yo7?NVXizcX{V+Xl4ph1E8Vh-Px4d1KRlaT#9>0UZoO zZtrUnu+yr8n!syWy4GNNEv)u7%BjYYJC8Gj;LD!d=BVRD-Z?o%t~_z9QV*u1Ie!Rp zM(Gr5H6>E*2?Pjv^YSk_`A?Q*qj1NnfqR-0}kvCwd`cI7csdg9SkQ+A) zHHFu7i){Z0gmt*ux{Y7oR}IEMYfYnpd658&Xw+kGa_m-O1Oq#p>MR`Hqf*1ZM5Pz} zge5hUy;Yj-LO^$AkRQK}ArT$j;8FpLM@#mzu)@;m6X1uP2Lvt`SMa1g0($SD?ufKC z8b)^@cf+7;sZB#hShsBH;8Fwt%9x>=>ygBJ7;{6+PFm0N%q`CR+f!sZ2o(Ol)2RLa z8+o<8^okG)35Ox%Tgvl73kvL;O?ZXf1=6_XMzXjM@>O8{O5N&)xwPvm2Y>aJWBjL# zGE3Nb-ZZE*9XOAYzP^Se*+U>@=PIw=Svc0NR&lkqfsN-=(7tFQ$pai%W~UYd&`p+r z<;E9N%Q9kaS9Dfi!a(k$!lu(vwjO%LeBVf+=P0qA1eI40(N?dZ0wC!nMKah?iI^Eg3W z=LfhTPhvfQ%}=m_3uZM-%&C#D#W`|OGh}nnlRE`DJe)g-wUN;uP+80LtN%#?*&1M7 zKikKsz{-M)H#k6zsF7w#dD{c>j2+w-JF47Wq>g9 zD75Pf)FQ9bv+v%St8#?paTUu^JWeOM54q{JIm;U})21qp3s!o7po=#Z-dk7@qr;;M z_<7vOMIT#G!Sqk{;_-Qff?HjmPzzy+xPQ$KfG~;$X;5YYGG84%lwY+gx^^9i#SDn+ zsWLx96baw=fHl+Z5~GGK-pXDU_sK!#aK`Xgq+Rutgwq68wRMJP0s!b^k|xKWHuG&p zR2RCV?QB_N&wZSA40xMEbnFjOc)@A6YRw0+;A--oEso+lF9jsic)+X?8wv`4kv-f^ zyDBQj?YSPh2YY98V_LkeaZWhiV3S)?9b+5ajX?)t%c8v5Q}yj_o}pbCSq>fni6csp z5{Q_8RaxVr+pWp#S~gaKbz4Dp692muxuC?nUle`F4$h@7YC>xn)2Q^TF8-TouOvGc zcv#j52pg4R-M1;7omz^UE6QyA7p3{HIVS4z7@_;wd2-Yr9Yw8e?D_$Js%cUthzD_@ z&JPOlR!C#sQol1V#iWiNIQyQZ=z$6_)mm=waup?(_P{pc)AQl-AzpPHn#a$v82tkk znMOm z8H9qiovBaURyw`pM*<2KQ`8L^ODqPvwz>N|6frl3diXGj2r9RY9?hp;Z6w7$PXZ2n zT^b}^PrDRJT3ZWihy|xNC(PH<5AE&~coaN|;zb}ANTD1Emh33Av_I^@NWga@9}&86 z!m?!3CTvzcCeq@5EhoG;{-NK&MFZoee$4Dc>kI(`(>#IqFz4G*flRNc3YWs^&1v995#DeIY9WG4{e>SPG7%0peyp?2tZ*yJazUOaI< z@8^_tS@2TptnOoCBq^v@4h;FMdIyBkJ;K^3-7dy81vTw7Tk)avlz_kq$ZJ}w_^E<{ z8183y-W2^Nkxv1_0c!pbX~jl-jZOYR9bnhOY?>(HGR;?w#tVIw;-P#*ADxl!tF*Er zOQ~*v6v%?!jAI!~BetK(fUfkZvO)o?$||ry%@1j9LzMOiGsOYrD}5mIPaB->X30?$ z!x{?DkHk&8`KC)kKX9jj0~ZU}Pg8ulm>mv6aMfL79ae1DJ(Z1uP$_*l&av0PKr(yq zvneHw7+QyIfnWoV?oW~=Jp^3dImRfFACvt8dceL3(<8tS!nqyY zIq7^FeekMT!i8c{XBji(1T#;j)XE3BR=YkWT|#8{xMqEpaNbwtRb*z7MD$e-YX9u^1uk+c#X z-I)>QZ$!AdGCw!#*kKJca$PjHk5Tw$?-*0)$~76WRXoy7IjDJl92}$FaHO&awGzvb zPa9u;n%FBGN^YejkASY{%2(sWyoq7U7vl?ljsj|h@z2Oc5Aof00@maizm^z&Tx%g# zs58_b&x(6jBNOJ`Ypn#H}YTsXrk7f+yr~^3e60RAHl; zK4N9qHoTE6^5j&tk^y+%aG( z){g=sZ3^meIp9FYObhwL)c>L@ej@X?*AFWNxdL=W(=+V?%1rGrn&d+3`Rsk|QaC5}*Dd;s* zFm!^PC=-}%53n^--nEBIQA;QxysaKT!~+|x*UeNnAMAvi_*geH3vL^ zcdjG;EmNtT^c8o1nMj8k%7VH3XFa7o!n|vL2})2Wjj)@ZvI`cG|J@;V1tBM6umZ3U zKPX+VJ(|I?FIY!~Zt51`rV@hAD&u-SICW^VlGpm)n~S%L@OdQs=X#GbcK34rH48w) zwdQj;w|*ThD7up4`Z$%GA|m*$Fz8&c~7gx!Uzc~e=3fKhawQM@FkwKxx#lYelf zM%O^SgMv0Ikf#3+L&|U}$p5C!-i+*8Mp5RY`@(9*zE>LpM|A)}DPYGycH9mq-0=6eecA?hCT-jCYIx=fU zw_ckkdE|U_x==0b0+NJz6|9p4JkH`w5Nvdgtbm~s**8~^XcM=X?!}tK*A5<##MM5D z1lwd{1anS1*)z%D9CkGobvfNV;(FcEYC>DHN%*3V3o;FFv*}B3EWkZeaL1AJPH@T?CTLuhcvMK?%isxZGuW7h+r z9YYIOs;L(zx4_EOpI)!%TO&kYGyqPI?h8OeiWS3WX9)$~X&Lt8m;~^n0s$K^HRIEU zi6_`)(8Po%ZyI&8!q~65su~!RgRGfX38Z)#Aok1LOOPQ&{6sqo>uvuJX&^`G4{Lc3 z@K<{voj*`K*PzV3h1OgG0K-sqQ%@_`d#}KwR{^5tJ~4qb$k6Ax<~CXh12%#*!&DhC zM(Vgb?>3sAh{|)3m20Ba#a01TvtXd>hab%5OVTbO^%h@_cz%my`z{`p)zB@Y+89fk zEc^{HlR8&{2hG-+?-&vHFWVdxI=xdo?mWm^^jM`WwQSOmRx1F3St9c0Rm_Ln6OUj2 zF#P-G&K*1UV;y#+p6tCCO~l` zpZGrH8ND)xNAHu)~GSFe?Cw3g2%crmld zhZ;Ah^Kke3Kcq?gwoq?peMMg0ooIBFKT_pa=VY-*Mm1`6CZ!v{0XlL%?5$fq=()6} zx^Ah!vv|o+e0wd(M_zI*2sSRtueYqKTOPG9YTqk$?BRIAV#lLjnGe?HL(h6PUA-}r zdUtZ~88EYXHZ|Qt`yessjf}%wOS%U{FoLY~yf?e%nJ+b#=Z)>yoguwN z-e7&f{4L+!${bg0bYsy#N#C_1`x5nZ4Y!w>RjQa%*u(-5XN{#bUNs#b(h1k zu2AwNCXr#~Aqd!W;Q}(A#-}mj)#+c;~&c z1`|aSa0mH?HA80h#0uC9 z#^)?_`k+FN5duzcanCg+ulh*cz}GfTGx5ZKcD z{ok50euTOk!WNcAYNOWj!FDo#$ZJQxantc$J;t%T8bHVouyZslIGPm*=5AaUlaM&b zo6f`qj+Pc4Gl=;y;2=0`02|-!!n?WsZk>k9pnvl5i@y(VuEBdN9F-hh?gff_btYRk z!Lt+}PJMqBI^Xq1fJpHl4jGaWzt@>_vfWXwmN?WQ5f$`WVtqmTt*+Duo0O(sXR7`T zrW&Nq)ZHpgS^M}1LGQ0kEqsf&^f`TvQ-N4B<3`DEtjPt8=BK;h0z7ss4qe2?!I%I+7lLKu|r(1{wM zu&>*Y7?lvZl2CnAU`@sbd<3PNPs5JzAUkkP7s>@7OXS_EyTcZoAaBU~p7kU0H8w-Y z?rTl`jS-GoX&`{t^Q03lb<5F* zm`a^6kfwXhZ9dE|0=&j6^R)cnO=+6r{OKa_xM0xqnsvYC-Lu|pLDNSLc;PS^nD67C z-wK+D=pfW4I^g4x?(8`+^JhyEFo~lM>FqGLYRTrjM(E8|rQPLdl1PgAzL=zPIX zRa(>irE_?o;|qttCaq;@1PB>-2d)@dYIHv2`(ME#EJLiN0$4(yQ$^|sm8tQL4Xu3h1l)-d=SsBhjJ{=Y>XrFU z{#ptsrD5S*?e*JWcFj!ImWzTz5|K5 zntb|-hWu4IC5rB+2&zK4z$QZH_Ep(u9|eOC`?`NT&~7syY8A~tUpcNue0yDjAXOBW zyATa$bLa)TBsWe30LG^1nDZtEGOEE;jDVEry3vWOtlA80F2#3uolza2CLbS+6KDL4 zs`h=ExTrAmtU1)m)$Wt2lKTC(B=z;BfnlS@hH^a$oUwTI_{^<^+$k_Cqn&N1OUpCXaqjyx&`NJwv`- zMO2ao|ENqM?pS}iL&7e{DC%oKA^ld#I&_9*R$4^80HZtM`u+9rr(C8>G^M=|FaWX<5>L&-#cbTnm%;@aj6DgI<|8&P~+-Y>Wb4 z03A|To`&-G3;jVKRDt~>lBBza-oZULxYG8@I9@D7c6F0vWfJMxoO&?hA-oGVZ*dP+ zW~LD#pO^Fv!G}EqA>X|qWATUHuNg1W@kknPxEZH#31N^JRb;I~u=`q%xEPkCOuruV zwFj*RI5fo>IoqUT`b!mke+%V@3}gA5dzk>DqZ!NOIH5ha(aHi_^ltydj{3RFVx3e> z8;sNT9*a-h3M%Xi-L_-Qvn$j1#6JNE;de%3z!mbemHDOp|GarccAPu`+~&7S7cTze z<`zE6@uNF-)mUyn>GqF!<>zgNgtu8*nO)k#Mu_a7$%|~GvAUgtPyI9O0e*&6_=^aO z6LwoqZA+Pm+C2V0Xq;)7_`YCx;B~gymg{N2{2~%b*HAOx`ij%04Mg)Vd$v;RU$%)H zJU$_J&E3FqtKWFE4-gt?S8j{`mQwo;%y6ZZ-FdBNvE_QffDZ6-Rx3icydwS*AQ7_* zL)j`@J*;WJ@bz^jmTdWM_ANU;evHB2t=xJwKp|EQD9o%&Y)ecrrXYO5yK;ypw^GLM za>DiceA2tI5 zn%}vFKJ3fd`P(TSQY^EcA$$Li`4!lex)#1X;g9}~DtAs7&(*q?-|aojXpAqN<#!xF z8QPzpS5b+N2zUiQRK4QUcLsfWLeASj=Z~!v-?RZZzl-Kxll>S|Y7n*TGJH0-+ER5J zP3~CzKbxRf(W}S#3SU+&5Up}PsV6wFB}%9gETtHR2rU`KMSaLNS&gHF|Doj!!TPu7 zRSz%WY84*P!i_~DQEF;iKgLAtzT*=#@599Hv^w^hJ;b{keWEn&l~|3E^{e4(eAr~o z*dFZUh=b3ce@2x9C&I zY_EJs=S`!#zi+YpZ6|)*{(-6VE^4Q>DO%Oxe|LzaA%iC#E2 z-AGdrjtesXNJL#<nsU#>vZ64;;VWntTBx_B;R;B`hqjk4y+)a&QVg55bNDPa z?ZYKjO}*=i+cV_xAr1T^X5{eMl6-DBxw2T2=va2t^R!#q@4xA~Z^Lj=;!55wmn`+e z?_-K(wYIR_reVAOI1t>;a}AGL95|GG%3KkH@pIH{e?>A$vrP*-sJ{NruJCE-DbFP7 zrFxEJK{5tUDw7+jGBH?!H&v;J{}-`S6xP5ZMtZ+>{}w^+Z^4prF@# zhWQF&adxenc3|(>jq}x2sK?+ZI>o2G*)H&Y9%V}T8SB#Bs8bhz!*rtO&S4Ef8TaN9 zk1CE4>u)8|oy2Nfbf8N(l|@#b>iKNY(TYkFdVUo6j80<$EYbrL4_z-c0Mf z-{@`b^K*Ir-YmU~5A$5b?e?2Yr-U#!G;SFeCQ}pSx7LY`-8B>`Manv7++?8l*DmR* zgSAn1LSw>q`)GqiEvkZX5@lg**$P`H!j1$1m}t-?3zGhOGmPBZK4~_N7vKAxzaX{6Z<_uR8*eW(0&Mle0n8d-$g-)&mng%ztPp4Q!mxu$V%k zPyBb=eQKkJB45TC>JA0i<#&6_wPTvP3`JPcJXqS>nw6#V9-wtet2$f7t(=>|j+mWe z;^|ALY!CWsq<6`t=HfR=x%j!&=V++h=%OubS*TUD!x?n~QNUocya>{vMoIC~ zTL{YZxKQ-+koSMT{;2B)7_aoG?c$g3FN70lSNahe@Hv%}@sR>szAiq`jiw6FPo%QC zh+~hBY@>;o=3Uw1eFEOBeivX}{+v>sD79{{c2e|}>!gpgiPeBS^3g_{zZ=(vY<^NZ z%6{)AuL(U_*bj}{q11i2M1iK&Yh3j0?dq+ zJO+&Q*oIhMAYR^|1z;&v0lKmRK~QqR4L{032tuf)!@PkQL*z-K)QvK#vmrhvOsqDT z`56IEqB{jxim)#>7@K|$OW5KD#~+&>20+Li)Ad=zHkw$!BAqWllP;$A6 zLXh^Y`|0HA2+bV*GllY16l_B;pcx z$CvVehobs5p>b|G=OG$}Kkg+WkVIWHuOvVva!BP_N7-2Y5fxa@xaNwSm73KJuX0Kx zRL2fS)a&S4+rGuV{(G;`R!|LDp#!sxtJ|r8r5+p?er?FD-;}I9J&=5!F=RpYvM8L*T4hxp*-M@;iv=bk$8=gZI+!%l}pelpv{n1 zXRIDXVF$ge5Xz_>jfp-4io#gcxZCE{^$WXTv%SNk0rW4CpU{ysaMEK;kj&uL9(x}Gh*fg!QgB;F zUUHWJHI?}Slt@HM?tTdg>ja|MV2dD?QvB(}i(j~n7tfk5($tV`FP zEfmr8c&0E+lsZNH59|3q22Ma=UAol`072iic>EaLp;sZjLWH8}Z`d$-;nBEtRgc+aVVEoqN^dcb{REAo7{5{bx; zF<=K_{J4flLr1yRbo}GyOppySEeOZE!RK>=;IpxhdI~)wThp}T*BATDHQeeRHM_lr z6u*D1>Uh1$_V8=I-x!1ICklhxGOrP_+sR;AIq57Hw*sCw6%HwtXjiybuKz0f#- z`k}jT|8P6D;Iw=Fqg6b-x~*nKA-A7^(76mGfexs%GnYoH>>}vn;RX5EoeTIrUD>Z} zK^(oyqm&hiNAt$5ugT{yyoEBe?dQ{aX0L5z%ztnj)krpq3aO2tX7&d;x&3x}_uh{W zU*Gn;tpGET{O#1GGiD^2(@M68m+!v)+SJffCjReYIKH^;nUqpmxwp#YwgbxEvqj!a z!^DrQLsjZx|2M3kblI2f@m62{e1iS7UmwQ*6t>Vmvg2=Cup105uI+_=(5N`t`9ZR& z|z7F2BH@ZBpO6uZ_e7xb69 zoVI{_PI6O!J9bQMRyFPKUMQG#?6_wAa`{+&l%0kLPA$|VTSG>dxC*}!Z=$2ZUZnKq zg=&JoeQyAo(u+4%4xWVYpypsDcc07jYj_XO?A`U_qq+?8f!-=P|Z?=p{k&qdn& z{o&H9%YkDx8jdLU)H!vTHxzz$dRip9CCe=782rjJ-OH^$NZ4C=_-YEFGI0ga>6?t3;5@?+!{d@iFk(53iA?e=z{#OR=*O>*wPF-)=XHg9d| zv8%{=_ezq8z7lM)-^zFJea_}y%F~K76L_&2(d;*}TbrZXfWCrj_W`MsBs#9lzPie4 z@YC?DvfY4N9NxJRdog!WMEoy@A}9NhnHJxoczlrR4GQ0KkGxV;$b9N}eh~)_f}j$X zZhpLWN7=5fz9ZY%I!V@WN)1nx3U8WQb2?8_4j}LR(?ORmR8z6I?#6M{{0EQ=|bl_=K<#%Xd8v$`Y7xRq@K#} zPj#=PrVf@Ew7fGD{a}sBFJ+FiBT8Z(1GylhUAuQDR|gohKa*)Rl zklHnFVsbA)d2x)%Q!k@Jc_xzqay|%P#=~ z%L)+zV%Wky#Wo;Q?W%rZpFu#IYY)h)#?diS`>YnZ&-*2i%uKtIu2HACR~L10D$G}_ z<=r>YuO)pC#tqysOgOLP-RvDRiz{8(wk-`8vDOVdQH;ig3pb~l5>;2G8qovA0n=we zfc#XxJd1;vlsj?;_pMd=5th=1E9PH+iEjx>~PiwLAe|EIJ1N04;Ud4KwU{BvR}n7?NdnC zO)U_F{hFgjdANKDy?_aL^zZpS*Y4YnkIzYKhb!L3C7hN1LK^eJhfRfMN zIXOE0L>$)E>z&}$*mhbD+>)h>U#DtaO5UMV)n0XjIgZ8TS)#g1_brX-pQyeOQ@gv- z+I;1M;v;?BzLa)g9pcx5{TChEr8x z?YboB>2H>MBC2|hNaXsx*(4ZaX7i*y&cQb&lCyM!+P=hb^2`bxN{0et=ow}t zx)d@~>w}C;O#C(7rz7(uWw4~2lBMgX;N5<6eaOP2(s#7~*SBl)s>czAjL(xn1+N@d zZ-0^9Fywm8 zCXlID1X=?t3C~OGp}eQqmG^F=T@hDhbV_GOEKs?C$H-FJKXdj3zb0a{Hq^(v<#X(6 zyQU$PqAcH$b~WQXrn=_{)3_fU2C-5`7^Fj3WBc{WT*+pj;>S(}nYnP5I#G%={8-wXPy zhhkUY!Q$R6e)a@{TmaMIT0U26cvhobn{obLCQ3`c-hn>i7zK|{k29wOKB=UkPBIp? zt6+MAG={T)Wa;9y`ugD=X45~=C&xc;gM)IeSl~ge6gx>5^aIVHG!-Qv*`My)r?iu4 z%BLb8gI}DjklI>{XK>;*(j^|3R`F?WQ*YW@t|J077uyX%2FK;QH#IK10pYSc+QzKV zp{hEDxhZJ9Cbf64(2@mJbmbLE;Ax|H%j5^<)EDrPU#K5@zFl38NeC#IBsPjWj?MyU z?-qv$4X0&anQMG*f~{A}50PpF_?mOEncrt-YI?#*c>G@3mn8VHw9{9lGpVe)Z8Xnj z`0ElE-4UHGPq6k!=+e;s+OW;!nT}WOzh;3p!?G+}+_smF*c5CRWe$6`r>V#DctElH zWjx<_$v^_nTIwNn`F(VQPqrFuo+>L$`+$7X`BfB$hw2HXN3^r`fs&hx z6TPzuplc^A3qB> zhV8pF_Z!@g5#I&Z;txJiYQ+*562QR##axm0ry}KbS!;=BCfwgY z@|9}QP~0hgs5*^^>Oewb|xZsNyY zD~_@p{6g5UJP;hz|GD<{nU#E57l|C$=0L$JgWyp&4wCz2&K#qIm~BjVU%4;%CP`{- zvi8INI;LF@ddS^ln0Ay#@N<%7Je6QP7^^JBzVqz(D7n=4Thj|!4FQjI4>KCei@*Ht z_wJF+aY)^pf-^Be^Ud%qmEjKS?cXf-rrDYd-$VMJ^_^0a*$V>VG>BmK{PcUe6zg11 z;y`Ogk}T)}QtfQu*rm<2ubY=3HhQCVf{1N28d4O;4E=g@MbglLE%zfUe0^jb3r^40 zwh=_p%tK99z7hYJ#(B*Bd*~Qzx=VDv(&jkn?rL{&s2EG=#>4n^@~HYg zj3hxxgcW>B?LP#I$G5TU8{EE8ZLzPkMr_s#1hj-3EksA)tGSAM9qBWzcF(1t>et@B ztT(*>S4kL9>=G`sG${3Gax*RPs1vU_N@%^~@owae)3Xik#)Qq->*#|1;@ zQf3!=6iOTF(QnuPo}9j#I$u8}S30n_ey6h|YRo^bJ8qonx9uxXSOTEP-ea7TGBj#W zMJ~6VqJ$X}!`r;kbJhuP_enpfw(+8bS(2JnJaEP*-Pe4h$o|2b=j&Had$n%ET#f5% zPcN+)C}kWoGYhI=@44B4y;9(QqjG_gzsMBv?JF%LBKe2U`G1-@7YmlCH57>~`A{;@ z!!SCkT{v6xgXdf}HJ2YNr?>&T{$Q|ZX3?{CN*gGf+g1UwBAl=Kah$Ot)!ZT9R#%s1 zo8e!C@ab6E-AdUVG@akB0JuIGiI?3OvQt$@Q%fZcJI|jG0Fa=yXrSGT{X%l-0oG)` z+wETQTTFbROV!fxJd&C!7|Jm>i1ei_URE8Ij>>xzBj81lIX01)F~Y-*>3upEs7q(x zKgLuSJ5;4XE;r0|QoQSHzZ`$x>d5&#Fj<9EPMLpm4t%rPfB5*cVm*B^x~VcMekM(rVw&R4;j7iy- z)z1kEG1MJn47h^f)q|l3Imm^I;*qt@H6QESbxC4X0)@`3s#idG)&_vz^Fe(7fW^$F z#Eg?dK$(Pa$tW5IFEQ!x?E9)vcns|C!fU*Y;u6ja^xH76?KI~`@XX(0)#Xv~HtIx7 zDw}b%eboLzv{AL=_EKkc%#G9=asla@Zb3ylpp&yI2Ut!t{z;eWrSmZhdNLma3GZr4 zenC;bJ-tUnKb#ZKxIAgELI`%Rg9^M=b(4%D|n#7d-23E9uY%mU@S-Xc54wbI% zs?pWem3pnMdk}Et7Y9o{$PbFO#OKu0KUk$$J-K|^`YfowbnQ?uYiNNaU1(SE3JUr4 z3J_{#H3)vo(D!@pHXSe)W`2AdEn4c^K(V9s&P^BH<6Q9lshd?Hk~OxOXYJ&`Y6<UDk;mZK zx}@+~G%1Asd%0KZ162&g?x1s(ufN;6l{gUomEp*J1-c@P&GPoK13*dg<_m&;A?b77 z0qC4QSu4*4`aftPPnB@^tI*l+U}I_DZM3uua+{Mujdx4I&h#(59&m zyp2a5*l)JP*gW!IrwgAfh|^>JHOamrzvOBIuKa#D3qW<>ih&1b~}M>ezdCCPHXz+1pyUJ>@B<^gHiEs z81M!CcXV(0i>q0kce}&O7e~%#T%H$0{{Tr7)Q_?kww^jO^L+yj*wHCLxW~_;gUTyAzJeO(?yQ|2WW*!XgPG>0> zdb(D4cb@q2cAO;Lx_iiwW4E)tb<>V z-Uo*VGBibhe~ah**5b%VTrunm_37am7Q42ceNgb-I4nJuV-B z*%jg3>=YqqE!xtH)iShv%4~Ih?7pjsUoYy&^i=s2@oE`CRDlqfZb(<_%(e{lwC_Q&-Yl@7W;f(J)G%By?mGd-a0tBR#*szi$5ndwTaq zL+ud~Px)yVzc1g~2iNR$Tv(5IKz*5;)Zr^d1SkVb^cmiNFg$KT!z09qInfFQEQQ3c zy+}UB)+A>QL2@w=VF}4DlacCyKv>(&P(gS%Ef|t0<5yJa)2@ubRHhl@d5dK@?*!(m z135%U-fRZb_7Lg_&kyRx4m<1YlhJf{#c*>2T)g@h?nI2cTr(&aVLg?|ygYk(z?(X9 zP*D%mQ)P>YT5aX#Pq7aV<6)>u`q0->ahc$7of_F-P46Oi(%FH6peKxQ>{D;JEJ*R_ zXK4P93Y}eaxdW_$f<82VaB))EE9Nh# zgD*w4gWJ)g_}_vC8tDhrSbDeDPWq)Soxj7qpdY;+>DyEEuK597ge8%G6c*#+JC?hG z*Ax;^#^CUBRE%sqXfLiw;Qs5Ftd<9Cx!#`0%NMIjuik5Y6#aVlZ1K=;9soYa{~D36 z7#Q=ri3&jex%_3O>;}dGD6x2yn0at!)g#>;dH*bQ=>i5j{&6yZ4V>lmespoBy>(dQ zeYaQvf!v4;I$V@mxYJNMg!^4Yvt=(r$ZkXG(Z53!mRHlm`^Cb za!kEXwvl}A{>LTmg6Zq8)VeE56D9*Yz7yWO-9Y3ImW;G#<+ZieZ&m=bP7FLh*TUwd zQcjD48wS1h@ds`E;~y)%z17xAGAB_ts$PuMJT#*zPL$eL$L~E7@O1-!DZMm+80oE& zPe{qsUTm#n6ng{VQv?F_GGGWOz`;u?UOfe2ho{Ixhb9*Zg<;g72I4?0QmZe&D|HW9 z>vFDQ)r_$Gy+qiU^8QKj)!C)2(#II7Hl(pnrkV?GXkd`=0{s{h>iLB&;ph6ri|Ae) zDGkegZ3#?_ez7xI(b`~oylML0E+3;@Af3%2Jf9YD*WmSTC7*C%*8hzX6TmxJ%K1f) z6z$yVp>YrPJ4^!Rh}C?D(WX9Uus2o3{z?{w?J&y)XvlK|ECToi9Q_vd-@P>DQESmh zKo4J2>SH=L9?(mIB+C3!nV;_o2`F;kT-4av2{pqA?%$BRUox~{m%cSs0qb2q4_S-> zr(6OYC0LJgZm|aFR~dLZ36ZU+7$bnYZ_)Xqqw{}pqJ2yZw$W@n*e*dY?h}jR>Q|zb zL+#qH`v$6vX#InQICpv?#&RuPS>qb#3&uM&q9Z9#5B)8q^qW&1hX2IGB2>sw}@ z9-BUjz0o=@st63)3b^KlZ(tVd5RbBjMR}2_MEl3!jtUbXW$I#pDHpNF?Ry{32DEq(F zZW%lL_To5Djq)K9sj)h3OB&6$NhC$<@fUk7GZ+tU-~psaIut_EJ~*jq z)i)RJmO43bcjR^LYIBC290W*P!(}{G)_uB_rIZq1C1wRCW;`YPWS#}zT^q&z$!Xx5 z<-IHvhGV0hiN6r3W!RNyEAkIq76NbA;0E`dOw?cDtq7fjkx}HfCXkOX(FM_ zvKL1TzcMFPCus0*UpL!NTP1s)k6L_m3muqBB^=RZt*jjS9d*AbUsq?K5AYF~b;x|$ zJ8A>>j62u5_RhA&F1v$awIHCn=-!LZ@XDMEck+#xPKSoez2jbFE#oQ-Z`Is>a!SnR zfk)6ty@TS_W1jQUSu)CDOX1F#z`Uy&24A`;@Ql9sDbrP#yQ^FqQmX_z2a-J5^$jDBnfAb1ePBI&TEeg|W`DBM&3;cAUB zbiP$`od42aX89i~9Cp2Z3@ykw@2zsH+W%DtKe&!4EbynMs4&cdJu%yn(I;Z`=ct=h zFh1bRv90n*zif*=0MrGU6OfNFfwUc2+A}ymg00S2mV%{?%=t*k%|&^4paspxj1Ho5 zuLFea@txvE0i#HLPW*-YPT%Ab?N%_ZPo_$9!I;Yug&wG9VtO9wW{P9RD{<$_Bk6K0 zhwiI;lqO#Y=pCW?kZ{#HoX677Y+}F70%IVczs&P3Gf8Ix3YOq~My$sO4g1Ab#W{H-glf7k>4zo7=h#F`P}@?+k93 zJhwFyozR^tL$0L z1jtCaeFEl!0}2t(wxxTU0JelM$0?b&1d7v`I@{b{$?2LWxY-vVRBQs7;|*-MO3akJ zWi*JKsBg$x2B1XD=QRXnLf&k!T0wkgYttEfhWDk?9cXdZOL9Z18^((R4Ia!W= ze=2o6$+Gc&{csQ^rc6+JzF?uG_p=-$FJgbdcS8nm?*n@Lf`?E`qNVv7!yD2_M8Hgi zg;9QMT$$~aUsy%mK7*u@WNh$t8Oqi%ALtFRm*ZGY)<_H&+gu|y1Gf9r;I9#cyGmVI zZ|kgbszvSvs%NXV;1-tFYEix}0r_aq3d;e}%DU~f>$|^8DHkCWS^-}#x}LZ*&dS{U zIutv!d(eb^ef#jPz1dEA7EUy;$f9##k^y9C`QtU#Kgl@{ol~PejQ<#BF8ODy|F>#E zMQkkK*78RJqUZht%+{KuT|n~oHWs-b6`2$DQNDxumjWv7a5oP`qxi6$Mapqwe#_kF z?g{Y@6@c6W5*API(b%==W{Y7V<2j~L0;vDiu$d@b@3m

    )-2iMn}T$5Jppg^laXb56j~ z@jvo=lpp+{mqN*0sUjb7^;yfvO?O#v5}5c>hg%XwL>@hP{v!k3DLLfcs8Y$&l4_dl z-G%;inD^RJG>^k$=1jdU*vT>ce19iXCe+e%Z^R52_=1WB=BlyZ|9;)-(0h6TXANpp z=COB4cXWqJmN+nn-RnUPL>17S?bx^?e@_5ABU+=LD)cQSf+vB z5Q^o`!)9|~K*s*&M0lGLbH{WJ*5b4z_tLPd08^H&8o#L9;%67^V4Z|`-JqeLO`mto z8>z;s)nQ15rq#SbJr=-f}>Vg0a8}MxMRI+-~KVAUIfP;C-Lm*vPtBp`oj$zx)EkvEOG;NO6P)-n` zyT`66x#D3Dnr4NCeYSfU?*fJUIvuZRn33DQ?TZFM%s{_Sy5^KJkagohGuH9WQIuz# zN}NE#4VA>NnoxY{*tnVQF@1JL8=9?fR_xM7h>+tm&&fAprPg~uD7z*`L(#x0|FKDZ zf6Qd@{&idIm=rrgG7{-eH@pUyAH$&QJ8W*QewLuuDXa0_Wro$3Xwtdc`=6-drtr&@ zO;q(AEfQU1%gZa-CRD7=iLt?;cJpQ_l9XO*4Y|MO=c?CT=A`MvR3P>lKU#j#Q=;VX zj39eKp+)fyb0uC&Vu`%bi#uB~$h=pgcyDMf6J&2kWe`2nDY;)=^A%R{T7q@Gftq57 z02P;+JzG3kqYT=hgB6{J4$x~4)BeE5o@bTk00R7pO+@t);BN(Kq7Dec{$3Hz?h<*g z@d&0rE2V$yUrcvEI!`s=?<7KU5y9sYaUVBJp*y4$1K^fOB}FO@MH~ zH2;moy^aKhN;;CaF(=;Uiy-t50s5c+@bCavNLHJ4#*jpG z9;$-d+(@wuB&9<8*BXE(9HfGj0O?MtW#Vl%P1@*4vop-#3u9r96Sqimffy4o^7xer z2TPJ;LKOk#3Awe9FdfNxA_YK5s+Vj&B%kEUS;H&IfJ^?Gbn>5J`eQYDPLP}E%?kHr zrabJ9Y##VWdj0d!pWixply8D5{ra=(q@D=00rC>R*r30I5a}=Wh4{tXEQRlq`k3eQ z=K;ZF5VlSVFuvknn*d(J_+U^8$=jSbaoC-URP)yV&nh196ZroWs|Zp8_zKn=qwjgc zXpfA^W>3gR7qNv?*dkz&;!)(2(wQGdZ|e15?#@x6t#DbGWORRm%YCZ~*O#V~G>!mU zM5Acw`>aTs7OkkFxej8Ze{hz;&~Ejr(-Y9rU-Ct_MogYSaq2C54U{6}#j(UFkCn;h zeTgBqsRr0PRU`MITcCgg6ds#G8urd(Q}1P#KNr}uuFnCDtuV+#bJOf? zM4?k3kFe*jhrOb~lTl!gIniiNdSY+h2HmTt2PU4^r#-FvYS{leUeZrf#Kqc<>|16H z969`ozRP}M)t`%Mz*?f?}T7CRnb#Zl-^w?#mX%Bt*>L>CNgq?80Y#@bS zM>mSln_0N`Qd^zQ+@PJ#QGd?iKOim} zon#9D%j$mIBe{>CEhlyew?`vYg1!A3p#!`y=kb|6Z?Bk6^^~1tP{WyL`kzFdWVDUbi`F-##J_xp%Fy zDpT^$<#qyb?$+KTIjeDa+-{-{ zIm~}=Dojpb(KqBe1d5EVLl1R|++O8KKYkpd7{{ZvPDO|5>ocHeQ`~l;Bi2!s%w~cy z!S(GnAwmX2XPoAjpF5z7W6P95Ny@~}#?o^jq$`Yfjv8bZ5Ne10I{0Y{ib$Xx%2#`= zD5Wp4m)Z?56Dvy--|8fo2XlOEUJst^w&+Y$qhyxwXl#*b4rh%9HOAOGxVse|EB6Ns z z$uMtUTPzo*h`-yt(mQnq$kH1NZLYkotCxGQa#n+t1&tqtb~m?kP2%r9VA-<6=ADo6 zullgk9Nt(znw)ansynTyF@bGHC z(!T$zsT5Y$^74(zqN`_SNDYSl6Iu(OMR0@6#Ru>8yXxi|Ha7U2g8EeCtpRucNwm+e zu{~vMQvRb3w+{YnteQC0TzouASroLZ6q&VhkyHI5Tyqe-RgMimz}e%hQD0h~%<&Ay=e&bE>L9f1?-=5FEfJxAh91)6R!|nKSyo>Q{vXg; z{f+0*EB0fB%7(sKACHogD{Q^J8ULUFNvux-uajyUTfTg8ll}|DlAgghu~@2?@u^W@ z`;*de2JZOFovl-{bIRhjo5NOV8aqK0%(p<9X^P`K-qD$pj~81*KxFM1JwASBk#pRB zpH08krwN++5dTwR;r)@4U+Qi~7wq@?uI0(~uy8Jwd}7ZtLK;J=Em5UKj6HzRttjn6 zTQ1lBoRvE&toT|IZs3a<(V^q9P2O2;RVH|~kQd!l%V>k!xjf42Y=I!HqNV%qHSX_h z6{(i_^1~3tJlX zDc(TFnJ$)}Q^xZ^0X&?*R?=p4N9H1DBDyjVvHgpeZf8uiGy#F)f4UEjWgBdCMH41B z-#@@kc1K#0{8rwap%bYsb*7THk)Bg#tZHTFrY%7i>uzB|Eh{Yj zbUO}ea2Q`OCDYQM%V5VaS5QOmAnvyKTEEg;&S|#EETBEfyR!ZIH2ko!U1oTo2mg;a zPCZHjp$dzL^xcIHxYGM#2UC#=sK-@I_2HhYa?jK_<%|UFbsOP5%+Jr1=lpD=JH1Fj z7R_Z8;Y0A{)#xkIc`zq7as*>gwSJM}g-tqI9O@f7aYIg#u!!5~)f-LX+^McMW^!G; zNAOVeNsw#epG16b*^e_GPofGrZJKYHNTF z#FDSyrhod?l6mF+REEf2kh4X(`_$be^qUc2pyT1pbg%Rm0n{LG0!JNyR-Ig|SWr4m zY(zF_|0sZBT1!`;ZKI$;4qmC+yQj<^EoiCUGnV|gLY3b$D5;r5p2o z>wme49@Ms6AeX$TN0Sb!L523a)-hsc@r^0S^=IeSd}c&f>tUj4D&zJ{XtIqL{?KI8 z1zM76*kKi-QZrU1?Ukh|^6KcZ?<-=el7_DW$09-~=4M8lS$7A1xv2q4fuf)z_O4MF ztIBPY2FmFDJxI}oTaRVO82P?xR@(pgTxmJumSE_yXSOhE<4x*{|nOj9+WSUM- zzre>agvse)u3NWc8$QI?1{5nl1!Az*KjF$~X7COOSKRb=f2upUQuj^K{b# zvt*I#_oF!Sm-vKgyca}xH65ip?}8d8v2E2>YitI_mEa}U^(im-_%0~?{`zJ2Ex#bF z@hFCW8WitJ&X;M*pw@_UMh}aGEkobB->BbCneOglfHQ#*agy5m=Y9 zR-fO3vLryLn4~}mwx~uX05RmAYNt4UPARm;SC0?EYBt8kjiKq}FNT|NIjLo-!4GDMN^?n;4ZWcGTT>9K zfpFI>+Z zQ?^&&{9XXT5LS~EJ^X;;jhALxQ)B78Tem7>_nVFCKNcsyN|KGDKi>b?0f12k#!n%K z-C^(dd_qNB=@(EI`O{4+y67Xl0vbnz6&9f?M?Mlod zx%!AMeJa89)#3Cdn>iU`tAgFo`fS770Gtn&gNiVWBJkOY0&C~pRCK(JTvBbwZ8h_M zzFe1EHhg0(|n=YTfZXK}Uu?tx9bKr{ZWtM+?cM))!p zT7NfR^FbY3tI;z1pQ8!;#u}Ka>=BHo^|OHNV?b|nb9|Wx$mxMV;CRvk{O!^e%G}_| z%L85Ec0Y93Irl+815+9*##{mPR^+Q-iHKvTC5zE4N*|5CHs~zvfKs&5Q#YhyjKDq6 zkTo;Zf3mDHsMY6~Mg8#+bib8l_ES{U219=nd3a;xb}wD#r!5F_DN5O3XuuWmwaM9> zGfF*66?(K*A(F#{Y>MWFNuXjzx95ErBH=8OYrQX%aQtt~yo$SUjhqq_?zR*c1eg|W zgiC70qi>q;+@pk%(cFFh5HYP|_`9hRA@RaV`8lqOBbh>uimc{NlJ-W2@!p`cpd%jx2D%mZ84bE`aMF&g-E@!q4boGagW;NEEH%y<~4p!cy zy?a3QyFfWvr$ z#2ir{bVY$)irb*g!dfq$r*Fg$$N~CjM-!z;u+OGDNB0-b33lG`X}!2KliNLG+T8tW z_2&p3+z#{r=mWA#fhSge8mx^at&&5f(x73(e6HqL`tR4yAYjQB$*vJyVx#qOV&@@B z!gaQpbuI2K+Qf8!ucjO9d{rGOym2z4palZ2rSm)ATqPs7F^ z1W^<|H#Y!s+Fp0CbQ-8o2=w~Sw2V6uVJ2#SWxKEYzvyE82|w_U1nyN+0V{Y{1m_q-+y5dG|!{luc@!%r8QwHhZ0 zq_ir>i*YH%_4Vl-;0JO=6Cg^T{}WMiTm))M*!lwhV+4;%^JNgx|4#!Pw%-BH6PEP3 zb7S9gq|qJCRpBbzqXE@n%CP!O4tWj+ffOEpI{sc&;{^+Gl(o zPmYZjckfU28K4F7=bq3P!;0_vjVY?>)>c}5D*#eDmwN*%Au)D;s4u%etkRXfnMiH# ze0yR{#XVwUgwh>AYYDz%5QKlxF0E7lLKR6Aq~jE=V;QvV$F)D%^=@o$pjEe)%GT3u zx*2#t2b{HiMEMoK)H-rfiB1V^V0xr|cNU;1Yriqc%C4P?)*GPtsniP^Hj%bEzHb+H z*F>g#d(C_fBs%ZL{O{)PpCuqDpkl_us|Wa6l`|jfj>9r$#$lQFx6y^_c{+7dXUYE; zy2M05whRK|Lys5ElMaZv|LPimsXV~+$!CTnCkq);3OWVfe*lqh3S4_~stnBpPz2h) z$zM=-W6sjc%hnh0v+c@yT4$jsS+A5V*n1SNFl0sqv{i4mHWe>Wn-S}wF~|K%3QIJd z^3?L)?rhB^86Lq96LYahWHJ$Nw;ULX`Kswa1-c(W%Tnyyse6-5)_o3`prQx1HzD{C zFViDmTchpoG+t8q6%(?)wKlhiaTX8Nd-UPs)0{YhUfrm$ub-bgdPwGNwW=Z|(-61a zFu${%#6LWf|MuTJQwdEz2%XW{XYwxgWVA&}cEqpF*Xz~#j9gMnk%TG}v0~d17n?Rd zMaiC*gVb^37@84s95dT1MQLd6*|AI`cc?9-+EA6)12u;QxQI#j+eu<#7+bhGwA z(8hJvmv=Xd-F$`}6UT*o9jXC!eJfOHFlbjKhgqAeNqpo4U{ch={`iI#eMmPHl5+I5|LYT*pYvv(oO>+gb?#yI zdPet{GjqJF`{1h^Y4;G$2BsFE=R0PrO>yr8(z&yEWmaSCT*t!l0sI|s2j3QKe@Dj) zOrTGj&Ms!*$^%}k4hm}giUxHYVn^-Nv^P*RPx6YAHs+K%uU{*ERX_YQU>Q6To)!g; zl-ZBk*4I=x&lOyC#)stXHMah!OSPN#S&LF~0;Qna+7oocd*7NR)9%&N zElhfdmIwCcp}rlB&uj9f#xMyQr%_aS0oEhnEe)GO4@>T%FK%&=lUC7E#0WtqFoN){ zr#gRrXtWOkW_o?#hfvm439I{)7BiVifoid&H3QqY}ZH)jnBhlZ(M z23ZC<2-opyvs8+Yv>cC{G{5hWX$XjNHG2YiBcIT3!7dQGIuY*>DFFmmPk2iydeIZtK|Nv8@aLFOtvDOyXjNd70(Z?bm>NFw$n|c( za${veZ@|%0jNSbdJOBXU&Fm~228g|0{}-K`?3!GWWDV7R;|w%y!|s+htK6{pcGsZ$ zLk+Wy&aZ-Hv4*=#TJF~^d^+MQ9@g)ONKJ!1dX{e3jGWfAC1TL9Q`a?nAOl-!&7H1* zw92kAifpMfi*Nrv?o2;8$C{H>oR{K~0I>KQ`qs3(>th0;T|saVX?xrZN3=?F$6s^`QHVdW?Jm_eN$gC z7aw}4uXJ=|NZGY$}kPS zn>B`SY1pqOh~oGnYmvM(bx@JfmeVXA<0ZhyARbePc=g+x#!KdCn_eOBf6 z>E0~ZD?iGQF@6)RPPqEu5uN22+46EdY+!$<2j}IjaP4ACd2TVBEn>T%3zJu2ws(zokv!z9$+`{Y$*1v#AQV`q{ zXHo9C*3XpUZtxQ;{C;tb*LTsybG>V^5LcvP);rJe+;<6;HNcFx)n>>InodS?^-9Cf z(D5v=xOEGX&DXwN8ZHwF-u%Ne0SNyh&jni|qJQ;S4_>47TAu4niq1pq-EWAycEwQ` zRB09LaIZ)u2O%GcaL00=-%bJR^m>e%)6Qmr9xh&Z?MT>0ify4Cft$tBdn3wjRLlC6 zIR%ng-0LCSpla7mr%>$HLXSm$-`FVO4W?T2FiL)1S9)*0E46Eh?FD>Hi1%Su=gguYSUK8<$(WxPIMV$V*a&A(rNL^ zUBLLLywgf>KA`CFTjP8k1|IO1V;5<+(&v^$Lm&Z0hJyz0FA3wuMX+YRnUQx+B>8$) zh;GP;yh8+%w2|yGK#8b#ZcZK|DOF?6f&1W_UQP3n)MuZmfN8Ke*iF(@=nxqO+-H>k zBgqGmstp3{EAuT5l9-p&m5W54rY|HYf2z4yGhW z?IbY6f|6C7y-d;!t5%$7O_YfI7JubW@Zz^FRGUBqz0~fNlK#e5oX863L5Xwoptg*^ zpDrZiDDY;XpQ*nJ9GCvpFiV&CVX*#l1tej+Lu4m`VHT9E>fa2r#EFJki8aOXmq>jx zDMg^l!fwlLla4DV`oVjEEY<5C@ca2YAA9L9DtMUGEe?@6UI#PRK7Q~tsY_}Nh~yv_ z!sD_C=}Q1(u?w%%zsy1E5-Fn9AB@TscR+;wdkTeI0$T@bLr#bAK?}`a*F2O1qZ(0e z`Hs}~1FW~fhseZzpdk4pI}hB$@ZX%$g-Kg?Z4zC##o_xyrh7*GU99(NTQP9cM9%B13Y%!`eX=iSPwN^4C!$D`BCkO03 z_tH4}ny~Y96h>UM1>~>x6~U_z&9GB6Y|S4F>djiYl%V|r!|9G*JEPRA(tptv4dYL;qhwAx7={ec~1z6GVUE%tk)8DU2x4Zgk3)pTxLF|sBR#8TN+l%Nu0ycUs3Hsy|jhC`=j0IsI zv7NWGjh0BV(|P^p%l;N9^{Y24-Md0Q?Cyg{I+eC>E{nU}5O8yDl{i^G9u^cya>!47 z`$$~;B6hdWM1gbr2-?@3w%6sXP>9HrXPNO8d&;!dOZ9bPr2N&t zsI%!3mby6A4ZT;)h&YaZxx+n>ue0Sy)cj=5rQEeu-|5?zZKNDs@_!@Y10B-5tvbT` zo}Mozdt5@dEry1Rj>j}CM=S4R?V3?{ zwe%6^@ubzR)sBl>PE84fySF#I=!hBk`8NOepX1Y!b9`(*4>RmgTs6}(pYmc4-?2zH zUt$X#{MPr#uW|h(qv>&{3r?IpbTwn6UM>-K!=)Y^e*G_pptyO~+p0J4+I!$!*6y`k zWv1560|a{TIy#wrreDn&fnroxd8H(v^7){diCb|E8mPj7M6jX`e_bbePJn+c1(@X$ zn4_!{2;(UcJu|(y>84PHL;;Oa{>>bX5eB|&E^zkCdcj5lceZ|3_H;)=qdL{T6le>z z6}>ECT6|PIRK@FKHYQGBe|ktR<^HR5{lT|9KC>5uUFMT*JiR+cfcx#1@i}6>>5zxwIiKF$r7osowqY_HSJ5Sag>4!*B*?c!u=N4Y83 zNd^&TUq2xZxKM&WVfyyebSe#VrS1)-SoI9W*z4QO|4?2N0aVSef@u}aW}Gp>kXY1} zl>i;4uw8Q8Xt8pRC)#CebsVP_kQPAxAwxkg$lf#jhw_?hiIkzdytxq-3G2>O;ifuf&TqW!W@`LGlMCc)<5F0DltzwLKDq`_UNX4LPA$Nq%XjA^puD~l zDgotXlZH}*^@tASqM7;d?bC0H6p=FN(m4%GsBa66c#qcJv9<`}|Xm>04$!MRJtAhWM^W2jtZ3?EGpF7NmothsPz8K_ZyB8HYl6vda zUJb7;=KNY&*K~Kfo~Tx6*rm}5SjUNSf|T*Ugfs;pNf;&1M*3F?|q(JMrc-088v+CQ42xGaDYXFY~{d;3vV7MJ#J(+7e zRTeqPUairsenG|N1!_NVF`=Wlk`P;x^jht`=jQ8st6Q`-34N;R^86>^`UapD0*T(; zn!BlkpThH)T1sj$By`=W42*GK9O05S^7^iBt;~kXGZOqcP}2?fQ9#|b8V1#{;a@`? z!oLauq$wMujBT+ZT6YalmfJiQ=W_|6a7Ej^+3?WH+tN1V(!EbcMwa$0U9}@+pcAx$ zwybs1I2tfe*s8?cl>;PI3=s~qxeZzm0UVaTQKhtW=;e+IO$g;u)zETTk8F++6;pR3 z&C&iD465BKGwH~--_z`U+K3+#;?s>AtRpjTOprv=`&%Pl?>|<$#m= z?(Rm@cfkzB7c{Q6yryax7UWjXaWq+e(bF7f#x5J|KIEm|fm*XA@@Y+zBSr+=QmDQ7 z&^X_%+J!l{&aBelwl!Y!b<<`mc-K&;MC6myAd%5l05IKj&S*O?0Eyzz7HV{*^4Z@N z?1o4|hLw|CDF$q0M};xK?nrLjSA1roZd~6-T&w&wM+*54M2 zlZrhLk%dyr0{1jQ>mb>bNq*Z`lT8U<+jEQ0uQhPfHf86B#A!Wx)(q;Au4ujpu+9UI zxV@w^JU^c%RJtvgeeKH|@?S};5xtqK*(;Fpm?DM}0E$ODQ3G!DV+MfxHP{=55X?aJ zS~Y)yS0H(3X&({xJm9{aC_cGZ)5U(@kk)f%X3{M|lYM__;;>G!2V&5-jD14Fy6uim zm0P6URCTFVz&K!9Bi3j;6cR@~?ee#B@{r3XDr}^*hZoYW8j8=x3p)>YzGv`@FuZE8 z1iZW4(e%&e_e`r>inbOmWmld?9hTC2CgOpr%2Awl1h*%kZ zh1GQ6$K6@6u%T>h5O6JHT|M6HgHJD>%Lup#PMp!Kj1@5ZAU+EW=VA}jRN00Y>_7M$ zE2Dh;S+ycoOz$thE-}yh8H$YDF+ApPivy5oKXB93!1i?YEWyQ+zWhD+aX(JUo#h1d z*2P*yk4pvPu}+ey-0PZ2^bAahzig92TU0$1GT{c}&!Cm1=7jAJDIn zQ-t>S_o5JcKv0$}gDPJq?94M<$xuwRt5mwC6oyR z%(;P#`h{OD(_EC1K=XB&7+ZzY?75o24F=7!;Vr@n@Uk&`x^Ilb+s{lC3Tlkw$KZZQ zW7UZTi^Yp)2C!*u?^_ zQUu7LICZxF7`8VF(sTMMW#apaZxqA+5<)HBt*c{mfbovmG5VEiP zMRvTVO_m?PNvGECOSr_HLf4|dXtRb73|EujdjlP`9GhQ&UzbKp!dsN?IfS(Xg<7a@ z!AtH~i(xAVk!LH7EEnPw*DnKS9P4PpX`$gr_svuj0!FoZ)9+!xUhXiT>xO{K@~x9R z|M+!%`^T^ACMaDNEg7*t={`e0Z~bC_F3mi*1uk)sb4OM+L(!^~dr3FrqMfe$+1!_- zAh?CkgtNq2&*?B11f1u)kr$qa&S-_uI-msTuO#vFj--Z_bKwQs?_Av{XYzO)#c6a+ zEmJAKeSPmtT+uhpdw{Ylv+YuDK|C=8;t30*2m<{}6_|EG(_ika_jq|C-D#^v#vLDi zuN&V$8q^w4B?JFf9Oq#W!$;gT_pe*ZXc|Le`5@P4#=hM8&6}b0rz?+Yaaxey6Vg~5 zW&e!c&4pXFO}u8BWf~~9N*%9X#m7II+3LDe6*%O2NMR1Xw8(H7;DlUuL;SoD%>fJi z5~BmLA+#-y!MNim#Is7*bB)8W2nY}4$B)OdVQR$23X|D4+`k^8R#%DUO4v=vI3loa zsJqwI4te(j=cO12E&6I#CSLPd;UFs%c!9g(z~Lf8dVyq@DRy=t2OgnVk%g8#~a>85tpT8&@&$z zenSgTtcF2R8P+)Es$KH>hQYVv^%9pX+0^}}tkDuv?6;Pd3&p++?Q zQ5$hUWt|YMdqvr4%&}5W!oGiC7@Xxw|e7eU2 z^M^X~eoU^WQ@>-?t*ukwl$Dd)VR=dWK^;VcYw6Sx!l`no?RqCj(V0UG#R+nl5m-#^ z{~U%uSyo4?zCk z6w9B~G+)3x&nD^#l$+$XFzKX$!yLR8N(D!r#2*H1+E|kAPoEcAnpj1V36ts1DduNu z%Q~*XIz#JOWfR3l*V)$bT2RW*Gozoy#k zLYoSywBnHN3H*Ne?8*PxItluNGlO<#kB-!6S-qNi5ab*EAz`@u#n;qFipBRVa*M0& zhA+aDbjz$+&E%t_w@#b40`PIEa>XPwCf|`#6sQn1)E`IL!!zDRf|^KWsq5EV%ugr3 zIvuP(bX0^wiDq%Aa5Wr8B(&iYt_@k1zzM_6YP_JtV2=Bpm`%fd;@Pn>hpH79oLKJc zMVb2Cipz3IY)OT%e#<+(yRm?{*G})6uGzkg30~Ym7~y5(O4}j27>?K4)h%aUoYlKK zrZemPh@;;h;-q!V9WMnp6vf|`JKysrW&}2DRJ)tyt)jp<70WAQfAYkMJAEtR<(0a7 zPhJ9BkuhJq>!^#gBKZiYSX`ZIPz(<00x-2 ze-(qdF74iN(e~Ry=1JfO4U-6Sl>9 z0>($Lfbx@2(R}slp(n6iLSP!P(`(%QLrbt2YmBIyJu)k=D^)QP7w?(&pi#5X8AS=( z;2g7JheYLG3BK4FO$>$lYcxNlsR!SdjXiYe{G{B&d+I(_QKtNnP2lU8NBix4r>B#6 zmY;y9iNMjuZ%~#C+Wt__R)=%3YkFJ0Sew!(^v19zwL#e6u!tI{yTUMjtN^|9Ds-RX zMD*Rdp`6K=8?M(WE-;IDHy!_y#JL_%@wJNl`|gI7`(`IqE>q%N6LVnrW=H@oJtIo3 z3CGGP%kv5oY1ktR+Z*2{+CErdRXl-T9PCl1Wl9P-jxQmyZ$QgmPk+`8bX^AP{k2X7 zr-67pmgP{w;LH6!UY^7Fdq)CW)j<05cX~rO!#d8Ujd84Vg2v8fQBDT0 zFE~br{cq}1Dpy53YEdpfzaCrYucL1yi)eP9PO1IU{aKDv7*tVmGW~lG7lM6Ctzx?}=S3AYTOx3a z{ zs-=~y6o*w4Hdbi5;lSTCi*ig>GtUq-OHkk?s^buZDpWPrX!H&!Hm&{)Q&IfBZBqy! z2HvK$47%D?Z0&iwR5&QQg|b1ASw;rolufeXEV*>I95wg0(JCk>EL~%kOy`F}t2GsP z-xE(?m+P!z@V?=mqiK-MhaM$`JjYeIevl!T>dhI}c8jeO z`58RxX$utmTWYL9&A?haT{D3pZvzgTUnp8tJ_5 zj;qbtH2(q8acg(P3s3P*NF?u*ue$8LAlf%~<(Ke!YB`{uA6B@o8S{2Z?)%9SYdwm& zjzl_pcF(Gy-s=o!KWOi~5pI}P?R=T0k|bf~{&xmgP+IA!&R#Cs$lOP)t;iQ>$v5Q@ z@GVt#u@wT$w)c0p6wAlOrFM6mrPEM^UAtq88SUsww)&j!5pvGcFzQCJ?PVGVX7O(I z`OZk0hWE_3LHXRV9e{MBF9+4)RPd)TKKGByT~+HcWPgcY?EB6%jNZS^xisWfx>g3x zHN0yj?MT6g`l2!DYABN0SG^W@YMGhKz!X!|>NX1M7j`sTuc_+I8rquOVl=IiM4<)Z zTzG$I5Y8a|;Yyewh1ZW?gwd*3O{j72g;Oeo)8#VXjeWIWR*ZKye`_|Go`OXWnmEJb ze$P_%yV~@E_E#l$iw(!HKJ`D`p87^`Yz6A`|8$yVpe@&@J4$J>4vf$!7;u|IhU5k1>TID|UV6_wq3J%?~qaO-ATRWNh+5l^R zd$3R(BrNX*^Hx8E6kGK)S67_F;Z@NBN;9?3cROyk5X1@1r#q3BY@c>J*H;h*OxC_- z;x@jGo<>x;Zjw&l9ulBAUhQvZC1bcNw~Y@tQ1P|g;oUg%hIcWbne!NrGfsWj_3>i~ zLWcZqcfR9N-0|7Q>Ag|HHDCr%Cq{Y;CTe1%L3ZlEIH-c!wm*c>pBraaF6fRC6EMXv zt@iDE(1~~yQ$&Nd&Sp({uK$gavyA^u$%#~!1=z^$dMPQ^ZoOR;{E2a-el(Py<6dhZ z_*pxp5cr3+U;sli>Qo2(!)Dlle;A;Ju3SwSHQ0S5a7D~^{*vc4Y&gpeUrpa_&(t)9 zXv0LFIyA9I6)_3ynSQR)_0Yc7?3DmDH-JAqaEd!r`P)8>xUVK{*Jn+qX_NuJ?-}`( z3!Xo3GCUqDwF`E^40ZJNYt_~~+NcD6ls;&LXX_(|JKWmIA-QjJ87j#~K%&}G&9n9B z7NJv;dk~$IG7oY??MyvzSSWX9OWE-_zb&&Oq6H=50s)J_0*bcDoZ;lsl#3u02jQDjy`L4WYd>MK}?!0531D1P& zi6#hR@L)PU+OVK8i_r7@9Q~L6I;LxK`lv?Hv|I+$j(BO{oC+xev#RWiPLaIk7k##4 z?>zSrf@~klus`J8md1>SAL8~(-nYZ%PSJ*r)oc&vVIFmCgL2iIz%5sQ;*lQhyrr#_ zTusDy*KEyQuOzxVeVngvEKh!Bf~RA4@-KENyia`y^|(@*Q%^7~fZUI|(U8;C|BdpP zo7emLS&oalBV`cSqGZ0xYV4c_J3gmMQ>e;iA#BvGDH)uMH@e~3RSI0v?gj`k72$&t zRWiA-l`qy+rYX+n%I7hGH+`;#Wr5ce^mRB5nlT7`2_H)twM8CuHte$pQshH0I`P51 zcJy>Jb?N3KxPHabI7{(zz(xPpIdA--`f^3gq_+L%-+-0RXd~m zTzzBfgXcO1gddL)#~X{xd>#@ft7vDUco0zdp0R%vC13jm;t93pssN9mGEg46b);SI5R>obIYN@(kRc2r_w9ZRuX~_xNjOj& z;P1W!Af%tphdf!`G`WY9WZGaN)=<^Ga?X1s!e)Oe_ScuDc#y_c$a{Gqe)M7 z=1lS5{$bo8ChQ-)1@7NWy69RL{= zTwQ(?v#MI7%c(bgpCIYCSM9yALCZ6C;qaSQ;vBRt;wC)@iO2HYHTpN7Csn|VT>t*? z-@Jdcfo&BdbvH?Fq_&3WT4jj*8c#BvlDZF$3`=|rKiWw0$gmLN0ZXg{3iZzq=vDe% zCmk^%f<^_WTNX%yM&GIdY4K4oU53<{96stzv^Plq%0v2!5kNdu8Q&r8%PYkJFe8|> zekDoNtXvR$#yA5iwCE3Krhi%^WH-ULq2A-4JV7#=n4VhQfU8mrL$ zh9sW#Pz!vB%gg}fKR5p4iJpHZvPH49xg;Otd(I#sx_qLL`+E!mq@MWj(ev+X zeek2KfSw3BLZG@4>pc6AATVeEMoISD??ANzS)_y}8sCU*c4Qi9-L&r(Xv19dqpKe>rr| zH;F$eNzCGZ`Si8mblED}6Gs2#(EY!D@z3|km(yRfG3luh4uL;8Y2}CT`%ho|U!r`< A@c;k- literal 0 HcmV?d00001 diff --git a/examples/grafana/screenshots/http-metrics.png b/examples/grafana/screenshots/http-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..fb6b167b5a86e2bc00b7ad605c87455bf20e232b GIT binary patch literal 311140 zcmeFZcUV(PyEhy_L_~@pq9{c~P^nU-gQ!#i>Agy?(jinq5CoClLAvzbOF(+>oe+9U zAoKvCe2e|;z2&^`dEWEvKfeD?t}9u?S~F{A?tA90zac*Uda`weZ5(r^|~lEmpYJws8~R?wUS@pAZ-{jCLwG7e1r@7;2@|Ih2iyf|zFFo9!Oo_faL4(s_jQWqIh~ z`Thkv#YOGX%G2vBMSe;G3U`Uqg6^+uN*pe+K2i2R@}Xs-A=bA9$!RhoN>%iYGHGbF z4MV1GENRBDSM`A?9`^?NGC~&#Ebu*%?2C7y6#ehl4W?pgv>}2ks^9NN-*`c~;N|x5 z)wLFKBC(EAgXR-U`I~Ijx3Rd`-UPAT#rtANBT)pAq+)m%jorFGjqHgeZ26Yhac7U_ zWM{Olr;13apokvRBl&{$Zk`71d<)^)m!-vDIQq8(F)@{j7e9p33M#iQx(@MQ+#6M$ zk%=Q2wGKglwfz)XcfaN1?&*Yg@v)1IUC#avjj6U`LYHhE|5qyVj9nZU-Y*QYYoNPw zjd9oMd>E;%Hgk}loN|(=8*$jaW4vxmNxpwCbYlFm_qeJY=-gNQ+LF)WDcbc7HPx<; zf03jNe@Au!0d+k@HaiSI=~V*WFJXc@mHlTD*9F8uveb{y{l(l=v042dsz#n&BvI)? z*dKQIqSyny@2l-&Na<6*@VfWv7R}Vc^(w-x&;HH1RC#?bAE54iS*#V9f0h!w{l&&- zK&FaaLi6c^g=N(Nf&-tEz3@W=4;zM`>w)OSSviX5i_$ZomvhWElmuIN#*UvED`D1# z%BrGe7k8To8AhzdKBoSN(7PiP*gFtP?nqwX^Sn#*B8Dom(A4L6h2Zd3VlrCv@N*IV z1G508Uj(|VMd1_JdAPYajEI+GUgf%Ckl0_)Qsu}WL_kHhgD{I+6|47ok&Guds~ z!|PvjSY$bvWRF>G>7}x|TkglYFcO^Ve^KQRX#9Asd$Mzq6ts*IJ8%V)Kv7yEF-)-P zV+huqdtGHEs>6wS|0M`LGtV%;?bPRoFT`X9N)Td{>)2I#SWrha7wQfQK9CA{NJT=XyxV_BDE;$}{xRT0&Q51kEh z0(NvGsSVc5=h`f+IuLY@;wXsv25!ouTd%2Jh%nv;x7^ozEGSM*{H=vrLi~A{w>S|t z8%bP7 z6Xma*zZ@F;$_=z8#1g)++uYp(IoaHx48hxec4Xm(tNl%7-r>mfl>D3s*#^07a!vG; zKvQ_*w!k*F7cZ_-fEYdvxJF8F=4gYJW|EfG&F<0G;#sZPGm_70WK z{*L5lad)Hw>7Rtrq)TTprdgzM$TKVOK26S6S2WMz=KL1-CQ*zjZA`vj4*4GTGdnvr z+vVq*+hs#9C4(*AyLxZX-pYP z!!m=E1A->391l62$zKKYkdu--kgITN4?VM&;`J8{)fQ&Sp!%Vc>0~O;kb=hizJ zM;hnPX>EFF@}lgm=^L|UGp50*YzGl|HJ8M+B%hqd&w62S?%}h|_FF-EeRFKrf{ueM zgV=)zC{n+b6UYJ!^R;oMOs-62K}Eq=t4~`T(3{Y-noa2-)7GUIb}`*C=veO9WWW3P zi?>MdHTd%!Cr0&<2Y&G5^??1rgTpoG06M*Yh;XETOeaOB&=UsYE@J<}6=PO3I2R-` zZj(5#-HhG*)gMW99{Sz)GU;xY!!ykf zbt`u%b*cOu(5m`1_AcjLh?+d(WK?XFIa9w{LuE;YWSP2&lJ&uidVid;madk8d%vI2 zqB(nO=FmzF(_oZkyk%|EVIYOss#&zDik{OXWw}!+*|c?$^#hT6^DkbprP{VI;W*%Y z6KxO;ZWa~GG^jF2F{oec4!!$0T&&=pi`5c0y-L=@EK3E3$5!=3Pq*mrDeXR2%vi__D1EkQ1E?HcJ>ub#>bBG zh`tG@kLMq|7q;4_TNoDNN!D_i!m` z$9r)+2r2&Y-80E*o+j5{Ww>N?xfQpRq*)U%%(=B8n4f;GYlsiI~(V9t@Z5pGWH^f?OK-?RMZ}+#h1()iy0w1 zwt6LcCc~xr-t`JF4n}>Dl9A;~;f zZZk~d7V7*SzA*T8oatPdk+DH_K~Y|P*Dyf0vo%@lO!LqVIkcaV-#rs$sVSq$tma|& z85x)psm_3>EU3-ts01||+gKR29+y_POBuncFp#-;#f+hlVryI5B~*(4+fea}QdLx`7Hrn$NPh>mL2sb()5u%4*B0-0y7{ zEx$*k)rMe)=sV|JK8}uw)%v_=gAuLk&_*sSN$}o=+nvm9Z;$XsKo_jzrWVRYZC?Mb9#<FueYXgox9N;!p@k%M<{>KxC~)JFj`$@?eZT?87cN*KL+2YL>aZ-B00kb|&+5(e-U z#CY&``7Op%5Y`{}F+m`IQ_!_P>&O7t%g=Y<7=5Wh1oP_#6(*Q0i}w6L<{brzucqXsWfzP!ysL;Xh; zdvgIA)pv5#qSm&C)LhK0%&atmH>s(q`E5TK@yfq``)70DF98}8dwUyR78WNbCuS#h zW@}qxmKQucJS?niENpB{Kn*547b|-`XC^DV$NzfCU+;NsXlGz+YGZF|ZAE?gUOjzl z2YUe;n#+#<{`_m4hR&wHd$O|ob6UUzSuUTjykKT!`TN~KQ~t}lymF?_h8AkCO)UYM z0euL*;9zCt|D(bG_2_q(SDLE+Zpy~P!*;dll}G<)Qzbh?TTyFEpi_In-wgY+@zsZa zHsoiyock48{EN_k+yy8tc$1&yZ>|a6>=@Cw2Qc!X>1$a<;0i3W%MWH9@bmOvSD=iU z1b$)vQyK&k21&dYQFO*YKyhM5mhQIgU}1iGO8xY%+?_k`Z@;NlMqmzsk@ez*nDQAC z;j>^Q#kJxesXu6_X&mEzDn64WorvGV$BjkwY%%tf@NM_oA!RnAm$-fUxnsHMC9GsT zH6mE4y+D{Z{`?W<$9*jkp%cOvP)=p;hxBat2rca!YvO}(M?-FCig?pDgrvdQ*Qcw& z=b!s`KBr0QkCk&*ee&-rAQA>)T>GDXs28q<5$T3hp@Oj9{$2gcOAPBTPzOvIU9D9$ zl38!*A{F2DBjOv|JEh3HFVB)?oCO+YiKgEC-QeH%G`Dewy3xa-p)i5;_momeZrs%$ zH`)HYRCFPg%7;a;Hg(%(D}MVyK#~o8oyVpd-Ajf4apXYfzn%zt3226u`CK7a9%ci` zMsfe)_Gcz7y+hS$ee7!s8q`OtgJhcHXMDJ*Bf)!9V*kOwLEgEqy+CNC-M6;nfAIo@ z8A?l}@m(OS@3TRD)U_K0I#Nik)xh7q|J#!A=?#F&R4VFMXhwYv<3b+M#8B!#a3`0V zh!B$u3+}k6s9E{M`AV&SKU8li5`eU(Rslx2>YQ+?0?m-F*o!{#>55*G-j5%tL!a67 z3#et-%KH722p>C;913s!nO(3@l~rS9GL`fZ0=V#9b3 zf^D~E)UxHv^AZHW%0pTG)M;Yx0zcLY<5gN;C{5K*E|gS`S8&>B7wLuy*M=Ijf9piS ztV12(<6YxAo~GY;<^l_1yQAG{pUJ0x52u$EFJ3tA4f8nj>kv@ea>!qP5FGZDr^plH z2HE60-tBfMF&*Z{5G^e=VhV%j*;qam9k-(BeE}YK-y9u(;_3a#wqF1xSyQ&2hnlZ@ z$ZIdRUmyR%F7R}Jn9w8s&Qm$njS;IkZkC7u!)~(frOritEzXClci=J$Rlk@jXP!S6 zlV;V}oUNBqFSmNwe)m^x%{e-T(=L>@xXa?L+Oke}6a$S|2qivJhB<{31+KK5R$d<} zcNfpMFFObC#2mozDfS^)kjOTdM-!04W0KgzeP5ByDQ%-w+{+qfN4A)*NX1RQ>c55!SivUYUX2mBAelE z!o5novh9HsEcq1u&hA<5>8+`ZXnxcTp10#7H*zF9T)tL@bQ0p340A*IDtA-z83o3c zo9=(y4=l^6*EM5= zWeTV3V}19K$9Cdo5Ltb~3Nu)vULxHZ)+EEX$yXMJ|DeG>lf_}l8;3H2d<>o2d~%T7 zz3>EYVI%@wKin@qd>l07>P z#B$Z0XA!xO;}r#ea_8(z3GabB}OMu z5qHDZz$ZpC6uRSGdl!iZiLz`A1Ihd@w#Hxzo_&-^wpwYbQqx&FJB;Gqn86oQ=PYYN zwc0}xW72$^WKOtyUzz&(8S0TlI9GnUWNcS#;p4RSThT6iMsY5!Un9eK9mr3Z#seY> zEHNp%6RO!iY>Wg9R_ygvCD`p&kQS`B+|?c5+kl#TE%hA#%+8^jRv>ug@KmKBbtHe! zPtAR=@a<4dHvQ}ewqjl3W5*Lu)&^SLAfCoU3NQ-)8D}fj7l!N zv{$lk9I|QO{`V^4zZEgwxk__!H;ZttdHBTpUFjm_Q!A@Rh zX(7r|NI_@A0CTsg&ELzrUZwX$6IH2pPj@_M2z zc3e{7biqpawFP&W)Q4iZjSx1meB)%aKsbvT1QZns$1fHz$nVWlExN z_9wK>w1%7!kj(W*ZF;au<=7R)gL!A_>@OxN@ z6iTq>SnW(><=N~cn%`j~;ttbcoWT8Uhn-ZV)QhAUQGL9k+K}Ln|B0fzs>SEjZ&No~ z)emn`JHNUkSwfmmR9mQjFqkHmAwBu)kY<(BbSOP_ev4KjEbjRR>GKGisMb5W zl`0#v`62v!(%S6n`GJ-=ev9O~g*r(5X9}6ArgpG~die~q`(UR&bNSq%(=L=CS9?DD z*XXCz^fC{fqQGLryp09icV^t3?%Ju6&;^C$vvjjCj5kL?cL!cXL_v z@yB3JI*!let71NM*yLKw*1Z=zvdNfnHc*fK0Oe0<4t?yOVNsoGqQQtXzmtqXS$?Z4Ve%mw2|$h!7c3*K>A7gU;e5i|YIFfB0F_h8WS&uRbhv#QepPqKhmTY}3-yQB3 z;^V%RkxSXGYRKL$IGk9hG)^q-4&J^`KUd&vIB>=B_BFzown$gc4k~?%`4DL z9rKTO;nqUk`f$H6cH`4z=G#RWMt>;gtM=f(H8K8tu=7;d3;OGw1%_k(!+g^8%|~+E zhn@7vp-BQKu}XE3GorXfki!l5PytlK2<>^!^E^XR=T_&m_k7j$iFj^vrq6ijPy4m` zKkPTBA~zMb!J~2q!P(ZVdv%ZWAf)*Lz63?G@w~EDwVRH1s!ST3VGYPzFEI2_ACq_; zkG^fb=oBwmBP&j=h%e;Zu0 zk^qHo996=s!mOZt9YPnU26tPR-7LiN?_Law<>nE&e#|obGF(dM@kSO1!%>R87<)n?U7Q)Pc@TB|Zvq?WvSgEOGaZ)l zzd@H;$#gjPrkqTkJm7L$XC!>msGajc&2*wDc5-DP#YaA6q4nmLqwdQBj6)UePWnw4 zV&P%?DR!yh$j_9%{fr>mOb|Mj-I(^XQVQR|5ATH`!@wh@R+(t#dg(+qdiCv@S#8k; zbrBu^7)%7Yqihm?b$u)>mpNCdR=zB7W2@gEcD*2gl;Li7t&bbG$VBPpoP#vRp&Z5T ziu?N-`z=#$ip_Ej)VC7HP5QwVu<=B)Sp9ophNEeuNcqO{QX%AgUqh>RwNFid#Z(vF z6FZ#B?YlOIUj!B^Gbn!;)3*<$-U!jlF#Mk1e+ zGC4=y6Fw&^?)GRw_v^8Mc7Q8IYSlW%nV$cXee;LLS5c>? z!9oDmTlw?H=;tapSPup~V@L>?wNt$kUVh}C$O&L>OcK%&$>l9At_UWnt-_(Qk;yh>T zfQRR3YaSzk9bM#Xy$C~*`JCH%-LlNK_63+*e~J3|bgw_%CM9*HzMLViQ0L;a{l?4~ zB2#A5PrjiJnT>M-3vPS2(M_eC;hH}usDJdbdNnD}sJ;-QO;5pFQA`-D@{d(=;LR4W zmsWF@LZ;03?6C_he5Ks6Ftf1>jY=yTnAPMR6H<1T2(!*6u5ale^y84FFb{YCm-CZM z8eAHWcH2C%qMj=~S1WNYk42%AwB&Q6Sjx-_mk5gbV|XUB`;t~N?RUSBuDP$bs?du1 zX>5*Ws2~e=ciO354m;{j#NPh!0oDLXmsHYmFat-{Zq;a9C%DJFJ_NPsx7Fq(xOif}o_vLo@k#UBttc{F6PRKg zC)x!wjJh&{-Q$>ux9f^D)`|ify1)gO{l0|r) zAGb@_hN$^W3Vl((`RVrY%TP{YCC~8*gD9j@;(_3RN|?(wH>R- zIUpnOi&P5Vix~B*TZ}12)4b6+$aRI-cjCwtLEiRCvRSPC^ujpgvgCa!a#Wt|Ndv8M z*q=s+%L+ag0PnN!5u8~Hr2P2RN`J@KRjyW{Ey^KSp?jhYRrBuh*AD}_Fy1Y9 zbJF7t&Mq|c>Fz{Yh!n&_R2pn1_j9}3$gO32#}-1`Gp@Isv&^O|Y|UQaX;rgMXF6I^ zf4bW(o4P86?db1!`yVcgs$ZBYw2cL;`V=??L99$7#{T+U^GaK=cx;nd|U%7wn5 z4OE-7NacBICBvy!vJ?ggv|M#t+tm@yx~X(if#ZYmu9Uc_IWKl)1am!O?YwqxM?kiMKtitKH9YD1#zk&OKD7dUn)d8RUS5Ph1 z3L1rs>{3eQ8!bfn0n8uFqf+p4A4M~ydtPW8380d97Bf4cPA~(bb?{GQ20=xv-+&Iw zQJ7AnMs${VsAd$)9#r@T=?)(0Aq9lF)Mp$JKZ^@LEd7X&#tuH=AFjT~LcIC99R4DW zz8kQ-*mFZ3a=yOH7-P_ex5Q!sy|IXwdI69?KHrr`txESzBM9w_JG@-bqt@fN%y1_f zav^xf62%bZX>l&F0crm9S=%L0ROm>T2Sfj0cx zM5kBb`cOvnE9|c^-z)vTZK7VQoX`8`50$C;cuh~TZ?aa%n14~#Vm7QAv9raQm@YGY z9uI-8RibtER$Lt(s5LE18fiHdLF+%x*0D%N_X+go0ZL!JUL)5TjN{}yOHd87lU2UY z*G5(%g;jg8eV`B0u-BX(5-0!Gn3|^9|CWLY@7a8z^YB*4u9}x?=&) z8fP=4z{mPfV-NIbz;*mM@pbv-UN`(P^PhCSdVsW=bC_={u=J5-NIY zB@-2yEoU`3lG!wDEmp!jUUa|WN!IT!@0`T%DJy2U(Kp1P;1YM5R{1G1vFNWHPnhnqP zB~GW6oA;}-dREWahqLJNeK7IN?E>#_EF9K^lIybQG?a{FLkAMtLWC^C>E3NjW-Zm- zWm10?$7O1nX>hYhHi=C;^IiO!g>gvTUSI95vlXltWVhP){e^*;_{skL4~KcJMv;Gp zDmO>Zv{e`{#KY)i^;Sq3l4NC)_)P(#6_RLp)k`c!$>*s)v+GfxKd&Oy^%O)(8J#o} z!lDI{_S}o!n1~wPY$b2O<*L+m>|wN$ou#Gr!A*gK9%^Tn~8VGi>qMU%2+z#By63aWn&Bgr$Bz#~xwx zf&;db&$T*7cj>~uu zID|&;ShjBmTH$_OmQ8t=v( z%#d!Jaecu2r-G*n6H|qw(Sv7I)C-!$?K<+hGj5cuj!7%?`$IlQMsjm$VCShdca!Un z&2P_($``nOWVgDg9sz+X3+k6Ajgz^1O{i-lX4M#p#4v*t>AMGgr1s>EtPUspt zJckyBs(Wko&z~tJt7E{^SL~JRA+JtQ$mN77QNra25#%=x42#733SL^#PHPjA4d*9P z6r4WSW4a@XH;HsB%jo}9Tr>W+Hd&;iuNU5+T+8|jdV6%1JI1?Vr`Jk`y!mvI+l!}Nkv%nPN(t(j_n{$$h><1-3AkMMrT zk@OaXrS>wMBGYQ4^{4_=tL|YgP(6pXgmtpZu12EsxDX<_{^q0~!5!hN&(=pvAtc`} z!}xh;rz9kqo;#nTFTIT39QEot!`!wK2Y9R9&AW9C5B>SJsuVJ%lMJYmEM|_%9U8>a zFT7?wQa&1E_XF^>Gs*^}f}+NY&O;0E0;vo}b$;2)SmBb(!ZVk%0~#5nbNmNmQfPwwUar&s zD0g*r7Z;RW24DD@G4_S6ddxKnRv(N__aH9KFhg`^{E#a*_wdx)$Q;8& zRG)UX@cE~q8V4~j+Z=pafVVR%*#l9@sJeY(OR>_g@dyYy>`bftI78>#=>0n9cf~@~ zI5(Xxe|z_1`I4fRQH&^6ksS#SZsIjJl+8-d{cO$Zh=)9`Edm<)e3y#+E#TVGOGQbK z@B{pxi9ZU42jd=WsF303!sqHgHd!y@k+=%JiyMpFoc6>ki*fnSk|e*?yl@@fYiTb5 z>shi4snw+GmtL#12{#>u)3#dkq+k0THnqnobgf}2kror%8Q|jkU7w$;;2P;Eh$_r+}eb;&~n&rCcZ_tk?v)9lwK=%yez`I4|ymuaunw3LI2 z%{c6{x}!~B+<)u|C0zp&!j!VBRPTGIl#0?)Q`6<~RT!Q+0S7cJRt>U0=%y1OSknDYt8T+Uj# zBWzh)B90>)@XT3u+o@`O^2w2VK(y3~ho4a_KX9eiaWe>Y_PNH#etk$oWCw@#P2e3s zuzvdei#nI1Z+86n|p|VoY)lxc^3iNh>*ef zlrsu2b%$!lL8U&V3{hP`RCm$ZxGZ4~^V$T-DbpP2YMSW3&8l6Mc7Ie}5b*+m`F{-2 zVn#Cvd!5;0Pg4G0b^fQfQojI(CmV#k*>2y97xeIjNQq%>|4R<*LGcR9Y1iI(&=oYx zZ(;ENED;?8Q+j+MuypHx>ioZ*ruP>j>PE`j)+9Iq|Icpz-N0TqfG`d_JPCQQ{^>gS zV>r|_*KQPvwM@5&{NwD+A0joB1%S2e!bJQ2+sXZp{P}VN0~2w3hwJ0_e=4B=AeQhF z9spH2MSW-gotF3~C7M^jl6qbD?jh|z_4NK=$N#eae_8*(yZ)bSxPOVrFE_luV)?s7B zcA=&7rT@fwu76IC>y;`kT3!vVGrbd6{Ah26N5VR1l21zsuZZ0RW}@f@*F22F+R_;& zu{^WyzT=DMZ;P<62$z%_COCqL4vTABTD78TP&NOC8qua3S1Rn#0CUq;HNEp&c6xW; z{ZZ&K>C16N2ME{Q@SRd2eAJlq<`4-ZW5JO~yE0(vIg*Z;9WI8(1_l5SVVqJ(NzbbM z8#5_9M+{I)2a`PhO4iW^`|=*f2S@MUdZ_Wq4c{RZ`bH1jB8yMIZwpspzgn*Yqa?hb z48J(841S*MMl0uKYruCPn^2;fUDi7p|8kLij!Co=Yy<3-NC#VrEAiMg^0 zfI$GPv6j3#FJDoBDlhMtVqKxiHr^kLfYQV1zc98BGsS+kKA2b+704QVg*8Xm0Bc|o zSLb!ZAE1iO)1m8ExIIsHBco8pOW<_tzc4o8kEQkCN#ho&9{GP61i%`|v;Q(Lf=jB% zrQ=`W_AeG9Nu5Lx_z3!682b{Kf(wdE@_!fX|9=a1yv@EXsIuE(T#MeHfZ{Uep!tA4 z(aYFH%1`5oOW>W$2G7T4ZWG-<2xtWGGw;d3$*2qXMXzhyxvuM`GU| zeyd2h|BmlakxJj$(X0 zwa6|S2Y7?t3XAGXF)qs~BFn_?#P`|yoG#l<3B_E)8Sr@9^HUd>g`EqD0Aj=5lQW$C zh%~Vf$)R)!!?rT)OU&{v)G0orStH-Tl4as%h`>oz1hZBX`kB zqN3sg&+Tv5{@RXTgfC^{&Rt#W;hMC%?bGfTC6=)B+%l8V4o#cjkFDk(^aS8E&T$;3 z?;3?NsRQE|C3$tacB(@neRx2NbY@LRp1Fy>>Qmi4AyOUF#haXZsv|e5aNJKHZ3>X9f zSh`lp*APA4wd>Kv!dOzx{=wW#(n)MGE4|4M_w$RN{V9Q7rwZ%CbNzE!9t0rY{4n^2 z6g(c56{4q@Mq4ETV2~CG15AoG!`^rfy(j<~D3xa!!K~47&2nSRl(sc6iVY4Ktg1kq zyGMVhVJ{6L=jrSLFn<^6im{)HE66OcozUeQiZ`oM!SkL%#U-u;mm@9!KXt-yc!-v<0*@5VIC5Ky@atFNwXQ7Ed2as5M z6dXT-zpeLm73MW~#o~%AA7D_2*o?#qWy$Y;bkcK?X0OTHFyk z?$8rSoC!iI!6iWf<5M9xcT~+mPTVKqEw5?-XG{v<{un}plEeT*?ooFhy_wd=oRttW z#^EC$F(5DhfgScjD#GB@m2`WuY-Ou8_PiDmg!Enfl&tS{QuO{G(#+4OIFVu$(C7y%Q_-zWm3 zzt6hAUf3U<5~Jj5$!SPh)+V|BOsz(DKrDvtU7Vtkv<7vy0&ER+_(sx8z@Tl~@N~j- zWx}{OiEh|fdY3Q6H^v#rzztROY8Z`^&<96h=?h$M;{kdiaDhL4LMpX&_oF(xqK>fe^Nx$U|qzsia-^E%J z$XgRwj#Pb9qFtt9&wq57;?o`jNAMc-Ow5k-*j^o{l z;wCR=HRiWImmb{scK9P?uM;%|4;#OUI}^R>Vb{K><9YTBzYZ*M$i zr|ON=kOu;nl&kF9P(k@r=li(}9T!VT2%$vxn01V$wg*qLVxjtU5Zu6<;Nb+a%wgKH zvYW3SCw4uyG_wPUz`pFZr!^l#r_aIP(4q`nXtiqWZI3$ij!)nRfPDGlg+W~JQ|bY2 z%fw2cdq)lg@ybY7V2>WZ#doCSQ7nBc;Ii{s<&^SvPRInm(|N*3fx@Kq#cV*#FP<7J z>AuTT9dGgDIv>Ql)>NPQ<%Eu(#%6DtFSfBrhk+IL_579vH|kpSIeM$mZNB=t2Gyo^R^d< ziQ@1+#-^ZSw{md$`?Qs?y?M>Cce&$eY*xhizCnn1$B*mx7Lzwi#y#AHfN5aZn`5JJh^vEW|b!{o}^`o~Uc+>~%9Vt1hs3ckBp2O*RK zX57nlg2zkjVLZcHyL&7`U1Rwg3CFXywF(v5D|;ODQyMBzHdF) zDX8zL!oSbam9XmVgU)z2ohe^nVfak3=%a4d z@?Vb9u;P^cI*NQTMRq#1A=Jio4X`T+2VUHyGq_8&YvM+}Z z{0jy=bqnvIFJvAE{Vg7%sckq>qumh&)KmV$w=PX7W34oS=g53by1kYD-Ei^=&#H9U zWD$VU47oy@&w}$G%Fw}i5dePD5IF8&7#cY~(Qu{$u|(i*!W&M9%$0R`vS_8EdTJce z;dxrMZv~5z&hL|w8WtL9Z=RV%{Q;{iF=UA4?+4=CODYWQX|b_Fc@4!*1MvC`$*6E? zCnGMkBog{>8t(l7m~+MkIF#e-bnw#HwhljnMMW(~vB1X8#d7YhojXUDa_ z90MZnOS_JQvC%`P0zpRT-B_*fkv>q zWSqroogM1Qu2IAvJQ!ouAxF3XxRv+nl^ePZ(i3|Km3ktra#tRbq4m);2q}$ckMcsW?03#*V3fgO`KBs~_{L$)4Yp zEkF5NU|Kfw|7C*BsS{s1xd&B%IKJK77B9R9oJW$5;da@*vtqNqV3F}WqXpW24kU*Y z+U+(Z9g`1%bb9pRCWi?EWi$mJeW}XBe>>@`Kz`Fk zWi5&`+_iZp+4XyL)0461wJRS)JPnpul;EB=&gOL@tq) zoce!9BfK&~x*n%o;~?7MN<&W{jUMh9eps=x5cJ$j879jc6N|j@U;) zmBuu<5p7()KorEJNfM6w@?ng`+B4Fq*FEI{)ak&iJVSataF9S0(r+q4eU>-}z%xQu zm>$GA%)pQpR%gn#D{dT96*}(qGP7HQojoktc@Ylbc4;Q?wV-n>*JIKi+$v)Bznph( zsXJBV=#8mLDh``;Tw5G{HA7OLk7PSVXfO2PW@ONP_BeRb)@EvIp9c2(xUPPoi?t*s zS5c#Lw}xB6Fii7Tg{>8q<2}|D^;Gp+2|NlbCg?#J0RSI=hC@hUQ!%U`@+L3-5f=Ul z->drTqrksmSafwAq5<@=d58FrW3V zzhNOp{~CKg2K|Pg$oHBvp+E$a@5>pmvO9oNkGJ!GWszcS&f8)k)aUCV4rE5eaAs-z zHtX5bV8l$hR+8JU8p*GLGiT8$1-jl#(77&w9`vMqg;_EG)p3{uPA-Aad0DR_$n94S zQS#&VX8Dx5e$nXOIDvOS#*-4Bwr+s8dd)70)yfa|5Z-WU#j3}?C?%zH>EflT2dns( zEPoBW&ijf-u--tKp&V|ryM%lEtELtOK=oI?>E<7$HkYlF7#=398zo$^QqC_OkF(06 zUtMY?9RN_SuGAp$TR!3apBU(c{CeqJxeFsD%gI_4i``-x-`o$lNwOTn=j385+EKY? zALa69Cszn~KGiXz93f_o){%Y_9#lDN6gSERAIqgP&c|!uZK;qgmG=@VPJ~5nJl5%Q z`U1)_Fsc@X=I1{U^(?U>!siI0;)dn~I>(cC2(maZcH`tBUsz7p87d!KpEwARC^cdv zuD>s0UBskQT;*~yiirK1Pt=W1=bOe2HT(I-zpB8R%k~VHvvI3{+ViLbMuxfE;I|J> zRK7oCCb%Ucq~ngh#Z|hG6YlJsv)@L&*Li%B_H=hg_Ske#1M>sq+J)_q1li1E5lCxZ z$eZrBm&c9b8&F?a>demCQe-0zphbDe&jeCY%r*@QG53EXD1JS);XE8YAWuIua2Uve zGt6q3k$UkMS&5I(ubL#wSS!yoEOM8AqhRDF8B5k08}=#-b7ju`lB)b*T%<5}EKh4N zhj|NZn^`MaeZ7L)>SU@W({rfnE5AVjKAcNEKr4B|fN<+Zf<#4f^~bg7m#;Kg#9DY1qej+i|)5R(rT$|R8Xr(!TA+OtC$juDs#xF82cNVz5`K8hv_R8aE9*iE4 zK5(kV448eC;}l3&nv=j-mXo+v)P7TrqYU%T;g3-n%F0*plP@yp(ES>V`BPNS8$0|6 zZY$zCniS}_hMH!c6khmag-h>^htgTj5i!9m;A%=L4f~OZ1VF*DxYmtNG8Z*itv};7 ztYN$0`%Of$0|1^U>QFgRgloHHtnC|w*3R$a>vMmoQFUk#fI}@fo8DXP2ggS(wueY9 z=VT-vAGT^F?(D>kvTZ}15)644wxZ#|Kk>=Ow(-18T4O9Gx45Tol~YPxoNu5KHgO<) zUDJ8>otH*}Z-^N>h0#2&1pB4X*SX9o$cHhqxY|8l%P-5f?ShRzbVh_d--ieZ(`;zO9 z2Xd)T(q%RY3ZZzlcn>(ZU$528y4Oeo=f0juMS=6PpAH5hfxyzpn?6?e4L63mh_RU; znzdGjds>PR2GcJRqHkwC$+iZ-(%iRS^6CENP!y1R`)=g=yVgg;H<}7_W#ifUuSu0X z*9Rghi5_KlEeCUg`0`;)`{ZZ9aYi2xZQxiPi$;U^+mQP!WiP-h#MSeD4|>W3>%~M_ zU*J1>X^jsW03zyvkfgm3ZxzI|I$8LGf8Y%1#HL4;6*u7hGqgY?#d>vNpj;9FtmAoJ zEk@3fvp!Mn=@iRj9@tZqbn<~v^m<4X7};sm*QE7%NV|{WZ~M;5JGZ!Sc0Ipxx${E6 z;HNr(9)_rS(J2pgMWaXS*`k^wSFm{AFqj%Qx`-0M~bD zQh$|iALU(hkQ<;qJU)K{|Nq!~udt@pCTv&`1VI56L`90AB3(g5IwD27^j?(Sk=~14 zklwpUlio`RO;LK08hS*U5C~EdAQ1SU1ad2)W9G z7?g1{22;?HoW^B3+L2C1Dm)QQ5w}0UKeIizb~nNH&+aW3xbZ&O7_}169dD8aqtj@2 zB#fk7{04UKbeYAqTNe|6N<(-z!71&TPx{+Ggni@HZ-6TuCM5ZUIQt;fCxEXM3iS{S9=-9`KDBimlD#UWn(MI(*?|<6Gw09%A>9eyt zsd*=eptS|$71PN|Cnzu3kLcH0h0$Oxn6@r<>pe!^Mx8*oWt2P<8Y&d9+mX;k9gy)A z51&%cZS2210n%g7i^+gPY#S%}7HdwR=R1^J8sH`D=u_9+DXNWfg!vST^iZih_cv*~ zqvWF71Pe%n#TvRimggWqF+pFBeu+ZphQDAp_@r3Z^lh}u>z`c@REYkxnhwy!`Yv@Q z3~;IR-i$uh#`VLft{4tG!vxXQnmlOuDn-TwM^9>l#@2$2<25_O;wz6EZWOlwr4=U@ zvweHEY;~}Z`kTt*A1sJRhQIRU*p&e6w0fb`fmJ~^pdhZRq?k7DS?%Q5xEYfg-d_W% zjH--~ptu|7B&lZ#v0dr#)6h{gqf?lgU1uo2de+CTSj!I9xq9js0pes&dbPT- zE+1L~@T#aM7}FoclMasttO~w$uRXzhdwJHRCf&kDV5O0uYF57})hV)Ja&MWU)F*yy z&}BinhFwGXb~JL$7)UBtod*01gnU=)G*=e73-;t>z)8ClIKUJmI*HRc2#sz-J7fwN z;eZ`RGhb;&KD_xG*=axcX0uX2%cV9jX_xn0Pvptaj|{CBTOlx&hD!HRgOVr9O3BEN z+Qd`w#=E^_^&X{|9Hg)8yT|-^$$b@FqJ=ja$&7#nIJANiaUuKhd=zr(t_roYag#gA z1P~gqfw{Gs{0^QsT6aG?+(>~9Rw;35;dqPwn|p3r1oGSO+61TBk5zbx!V_MH!Ju(dmIP|CXm-Yl-BYmn@3)OY(KkPl#bJIVoLzDOSFhV)*L3(-4-P`5W1Y zqF;VLSB=>zy$&ad!$(orgUc41cUF^S^Bv#K>|D)5IWXtj`yeT(zN zdk5yPf>+2^FlHLq)6%cN{)-p~M~~aH==3bP=+5si){;Abirs?iQaez3jVC9-vO~1> zOXNON+#4f5Svmm_!%^e2Irc2Rq>@b*_kgIsyU*2^D$MGA|9iKtdJ$`u|F<^VZGtLZ zpCX~y{kWy)^!e4~>9yo;p6OpR5w=!mZ@SndX9kN(q*;SfhdaY{;6Qk(!x;)L_^|Ln_f<2Mf?ddrS%59XZk0U03 zEI~v?gl)GlH`1n`#^X+_VfWP-IEzeZ!VRzi#5^hej^F$4G`^`ZP2aR<73Y2%`B)Gs zz$qpE(u6A*QaQiwEDElq8aJfFmX^ikedLXfBvd2$XH{CG%qA&mX+cerRr#=2l#D=) zndw#k%x`AfnQ#S3QCEWf)`niPv`q<)taGT!uaa&$W_8W|@JgqKLL#5XOTIl&fMp2T zlK6dvvJ}~DR#&Z2ZGFPNIoH<1qUommfdHMaX(JCQk4@eD%5&}G%SNzQz|s~*q~GoP zhm}YI%Z5$r$ol-sVB16F9Pn|+9lNOiC;6+?|qe{ zv6P4S>XuD03c-uN<@Uyy#C`$?L zjmJ4vHs7_4i{guL#iOSwrQRDE$I)SJ`bssG7xLHMYmS8@Fqkxi5936sWuJrfj!e<0 z7GKvC%`<<@5{dOx^&7bRHay<&1V!xyPS{!osz`qI1}e8;f_QWN2&`1I^(Y&fW69yz=$zOOwaj0PL zrB$pCz5l^3iS=-+N2p*moFxfB@sh?UxLwUE(BbvwHzFB|=LBoFyMEh=DMII0!viWz z9liJ_Hw#Qcw{|0$CIU~>-LCpH{iQog%=rPMDD}k6@w&}TWdy}n2~A~JeQ1C)bFT?1CZPw#kiD!aVDCH0R@AqW9x@6)Z zZsCe=Mw*e{s+f%;#}I*TJEvwOA#>_h2_3 zEO9V;BKb`{SBVO$^PZj)_Wl}JHs$8%&>GL_4L3Mm^=uKT*q#CzRGN|AmQfK$6}ql` zo$9_M;QyS{sJ!{@#9rCN6P%YKjbuY~7yOXY9DJK~kUIrk{`=O+mDUqOh3AVAOPH%! z6An$AVRPZC*VU`nOwJAwO`%htrSy!s!B=_`ZZifw3+Y)gA?`&N9#sFz9d%SAFnP#?NV z=77W`aSZ9y`SZ+LXQyTt5WcaW0eg>-5txM@_=8;TMt1MIN59&ZckBW52WuVTaZC?8 zjaIRqUT=!v?22pwnO6Qo&1&q=2Am*ecQM^G=l=e8@)CIg)dP z$D*-+j8+ky@aocQ|C3&Gk+MoVbDKP3ZVbogYtEkgeeccu__m?~wHyI@-85GYjA4YF zcy0?TV#eH6_xbAY4+_lgti7`2TZT8QYff(^>0sKQU@6Z0Fj(=L@6xPhpB=g#T~ri& zQ(QOp(MpElgmO-wy-FJzKvQ@;7qnj3>G{~)JDLmP{`<8qE%tk1T$#k}vvrMoI+B1W z_^`i_%>SIhT=Q)H%C>GltN7BncdkFDMGyybZz9yWJ*zsG|Tx&yoR>4?w6lGLg zu*Y8NQ=$)ghbY|M6UgDg%0`cXaTja4|D3N(S8S0Pio;KZVBjg|c+IIhQr>}6ShK7e zANTUK&dOfF;Sd$?vnMSB4V$)UIA#izmsbAKTQM;Z6}L9li9Nv$t}N)S#^lY?R{=sM zQtq5ZqV+bvqOeXec|FZAV9Yr9ZHELV3d_rHw-hRdQn-J>{)}wO)Cmeq61J#=apB2_t z(Jqgc0;w* z7Y~ioa8zHqLqF&-Vt?800B9^g7LiSrx@kp6j_HhP>2aBT zyBASCGW)F36p+042MXGbxrB{FxKx+jSigz_o&(5Vi%z1p%E#eNjwN>R^<-<7(R9#o z{g^Vv=e_Qvs>aEqZwA71PvrxD{E^Wu~(2AMcZN&+1N8Bds(` zRIX!;;tL*feoOuEu{V78&k?C&#&zCpL9q&*zKdBj_NiA&UZS4}aN+)Z0_th&_@hyd z_bbanfo2hxdDjkR6NkexT23S-s~Ph=l(Nw%nKjri(r-#ysla^1F}yzTY1roh*)W6$ z>6%wbyt^&vxVCjwS-1D3XvqO!@yD}ERDLADZ18JOkir`PaJ z7tx*}=;V?6$sHA7fM3?1;zP2}fv)pgGjIkni!Hza?$ux)@Df2ufdx0U}6yj(#<52mi!IEx0qK`ktoWZL|)hf6;#G zYaS4zdcHRNe4ir$fqLXqx()2y0Q!M*W0q+G$pO2=wC`cLpwKmEFP(rDqG|$#Q27aN zpWH;T=7=QEu7yVCxbZ{7Ql~J`QEkQOMMzcF)lYZl@6yocC?{J5C;#w#2WQ!O^A0oA zN9R~Cr+rJ{OR?@|#zl|A{!fXPeX0CdWbIt=r2Wl5#jk-qzUv7#aC}8V8N~OUVozNf z=-aWCZaToeO!{{3o4{51BpnY4pr4%5-IvK+c>gh8qD8PE`a`OU2O#WRNQ$Q>tR+4j z@P4BxMF=^1tl2X3xjyl0xPSW%9|Vu@$nqyzhB{nQWf_-5P8UuXQ@H90FNu1)kErr_ za2G$ZW?MT=DWmNvuIR5OCOr|V#$T%eEQjsTM`eXHc8 z&>B>zY+|xhlTOWuE;V3WX+&|1YvC&z9OtV3pCdX8E)w@$ThcO;MOa&x5vdd#ASGaHtV1-j4ik{a8P0~~8Uo*6pAXHnRgiJU;{TV z{+?_qFvcIIV^DdLy6J*VNb$jY-h_{ShFGw_q$k>IvTG9d1I$q{Yv#w6(USOvZUNQ0 z#;gpNlp`sM;_zTEJgfZELvI$VyCA%O7eqI1$*47C#1Q-*NP10)qf(1Mx(I(;<=gBt zw7Q`m(!RgG$WSp#5jQcGAkjO$5+YV)wLiL~oFTJzC#fV)yQvQ1V8c_feeMUw)v@4* zX>UrnUV)5U|M%VQ>)tIN#H8i-l4wZ|T`mgRlooofJ-!m@WS|0j%7fjbcv4JjJXv#? zV3g}HhB>_`mmK6D{#Bw$jOKRry>uaHKvT(0mAk!HsGC(RR~m$*5_ycmzYRgWFw#;PbU7?QrQxuw_Yph&Me-ZIU<_c@g>d1fKm z@@6;78)QA7OXV@2jk%5@WyC|s_H?0gRU5t5jYdZ}1Fy~8)xqaA=SukCcCayx`N3r_ z^gClJt@@g)>EWQ^tDqZr_ZZ zPQ92Gso>TdHP7IcyUiq*ijm3v0-i#jP)-?Ow^;c4z(H8u0g7)9z}-jrAUnJ>f*VVg z)N_@SWK|)@ulFkde)8Z6-Na8fYQ%%;Wcx~keO<0NcEz+`)@Rvi{rl=gHqD#}AAUY0 zMJdeoq2U1r5e~Y_ryAJQ3f|a0_#_~utHLQ{!XDhzglQn(^yL9OsiQgU`QHBb4a2Kq zn09ZSC%Y6gLZMcf?icg}UVmj3e8F44^N9m&t|O_i>6IuI0DX!2pXZ!owAewcy0X1tg8d#c)a^ z&jXsKZ*LE-m$@O%FD_->@(bP+S&wmDTG=zo8m41azGSKuzO)w%UG}O$(wnyk7`IlE zZ{EJ*#~X$2x$iZuQo3f0R*DajlrCx1YtGNv%{S}vZrW{_vn=5CJ|t);YCM$ex~}^b z2%FbD+=eTR-8us9<}e~*WZx`*?${+n*h z2CzZfkq@jd$Z1^6U;9FZ{4g1dP&j7T$Z^PF~Jl}nQH-+z?sHK%O1I_;-@90vzWzn6ON_#L$Ow06<$G%#JWxzlx=1c-U}lo0V`r#s=kF5a7-y%LWowohCd zX$Kod1zmr{zY3PS&kA_or=9-bDd3Ph%Il1?;Dp zQla%1?+`cKySwZg1IB!{N5`|S{3gqqIh?AthpT>Mo~IIVo-4JwE6OgCY zSAWl+fRID_3O`lo&n(!C6s66%f9`^?MgFA!1ikal&ntOanZ8zZWh3S%e}r?Wv-a`R zTwUx<23$c{Qz3fv!mpgdtny87rNG5dbVDrnW0+1luST)mwf=9)0QR-3h##704gZV~ z8`DZ3&uVGDx7OaqSaN6fpUyk8H14l`;B=0GQ_z&Z6+HM02ZEpHo|dDme-OhlLI&J4 zewxyAp!k_vPWKn_U|1Td3Kvu*yso`~MS{sIk&Sj!Efnf7OA+!(!?=leWrY}jsX_|Tas7LCwsO%+WjYc{Wc*;RN>{3f1+Reh@>5XO$br2@_r1xA2dP+o zR|68vRJG^#mW|y1FEzy3XP^lIBt&m!QompLM_N?ug}Y9ptKhdIZA~#VuNWc`UD zpE3VqL4o+}L-!xjoZ>IIlhAS~DF5U9V3r&Q7<#+s3?%J!SMFr5OhOC+;NhI`A7xin zRXyqFxx)C%F{o;<2HBRCAYZWWc&p0AvS+W1SE)CS?NoW(`dd?V^);@E11>L(X=CMY z1>0RbIK1=!STME|fJPn{33j&qwj(F`;K7fE2ae1GL|L!5@(0RTjHsnJcgCa(Z+NdZ zTzUHLJ+r3JQw>$22RsA917}H}pKVN>cPJMY<`af`ox78KC>AcT87IiMsn4CVlIHc# zRQNFh8p63{cIr2O>97L530NUQ#4F2FZ~oS-H4`dFd6+&V8x3Wy{OjC4)49cRPmJOtjaOrT+RU;d!U1db($7%?{KVe?%@!f#3q1 z-B3AFf9dcfUL1qekx4)PGiyr9^v>g~b&w_2<;FnVt4LJXmMCaL{jUo}J? zQBWXo?QdZlJ5l?v=8dy|X~hUogv!otWTV5-bARjV0*O0SE~S&U`qL%HT|l6L zVFTNlzm0JP7(+7h!r#WAb&4ubZ!Vrr{o6!DoZqf`37p6doVp=N_uET>!VU$V#of_kIh6N`0HQNJBbGHG`!XSw);CtV2l^cxBdw` zR1bN+UDeJFG;9$5+eD-~&2k$Kr{oEss?YEI6-<|j+lPmf&;Dii|GDP>ToZTR|L27tZ#}<;5ZM0%B0ERko<@GfracB%-J5sOMlVDdn%~O&MfLQX1_I5yA_g@H+Y40Q+Z}<5JaMwQm<0wH?)HoBUB>$%gklZ)>eh{J+ z;?>N+GS{5f9UYeQU^l*7Xd+D_Etj6bbilTH(az8JB3lm_1+$Ar8M?bTtDElLx^;`! zy<5XxPw!q%livkgxzBsQxuRJllYG=;b^MW+95QyeD7af3^G$(G^?g=73T=B&e5r-K zdR`9e_${~p+%++=n)*9)Uxrxyb|zloMApq*RAi94J%gS9W2=<)5jr2NAC>$a1*jyx z1)s!WR94x4helwRzCuFxT;h$7)QQ4=mQUX(5kS=q7p??KSdg1p5I~~|cTGaNQ)`Lo zfJIpAx&igg5%jQqxTrVO|BulKh{`MQcpJU4&-&c<51m0pdJzY>*UkQoj&X-ci+Bag z;3>7Wxwle5nWdQMJW%gt3x;Bzf}QzViTplwcH@;F&%D<3ElBMN?#|52oQGH%M}2-y zwEJ_`!`Hn?tEjHh)+HKf5v0*8V_f3lyZ5#pw>eBXM57RgaqH*&pRCSa^l1(x$ufi{ zgZJ^AI2+6Is?exGGDk7>s&_`i9xB~u@4*+c*lhRJy5i#Ot^-$Wiip?KLZgZS(a>~o zc$#U>l(L&N<=vOf)8F<^6!QzdEK+GaZWACrQmCqr^YwuJGKmGAovHp^y)QN6WBKiq zoA7AUws5aiqe7i0OJ5Jp1Ii!`u?Dvu^pA}#&}tu45#?c3Ns9>=_k!`mFF>V3?hMqb zo&#^F`CvC$Yn0QMzzN;X5}nV~t%w`gYCZ4lhm|SDqHN(17wCS2dlgXC^&&1?;ZUCz z?quty(qWHMKfdrYvplLy$97dnVSn0BOjNwM(sZ`t-*qJZVUl&X?Qa##b<(i9EMl*N z5w>;6jt4y8{zBijF_AS5ob}tGkVr`P+L}vE0Zga%2V^Zb-50b`W58Zk++mpSoG&Kq zr58SoBnU^jghEXjMKL9FUxPKIaX}gcw?6{uZl^y3i^*y0CuU~Hoabed`+v1*G1$~8&XyA6m z)0QZYsy%qLwh{de4lQnlsRm=X>|C6FT~_rJ9Wua!+u$ybA=Jak&8Kzb(7y>;he0kC zr(@uIeWHiY?M6#`hZ(l^3kFI*o;GsXYDKS3%$2n^%-UC?0DF0Q`~$w-uvU38^+ zI>AR9II^e+W((a<4xlhZ%r7t3jg$sH=x)nA6$-T+)p@%;pv zQe!dnwW7NL)nL9zTCduU1+3m3sbeuKsuo(?SSi2)Mt-7VViX2)-1df&osE2#E;gw|l?FSy zLDHlb=!J8qcIFPYKUV}c7)~V5MZQXRdNJN;n#^f%S7mE>G-28uzApltz66f&}8mIdi#`GtA5{Iw+&Bi4<_OjWPquN;f!wKkY2ip#8b zsc=sH(VF*&Tg8(pL3XTo?m=_{TlVegKLm}WDvo>*ar`%lh08b4=1HB}Rc`Lfn_ZL} zQ`;={+y`xp*B{_u&K{#od!MQQ&r5uFCf7r;X{&u=hs{-=pwo+$ferS_6dONSR5}4} znosj_M4eh3KUfZN%r@&N_*@u>1*^mdieyPj{PK;}?;q?fkm#j*WB`86Nbxq61k-aO z5%TUHGVqcb`1NqngLkltO?P(e^9&I}s|%T-8$g%u9eR;2Z9UW!i3XEi^YhwiF8rzI zOLMUe>;u@5ahZsNw!|cCkvrY*x=PaO@+K#d5=Z40x83$wA_Z_irf6bd zG10?m=aObI$7wsSwP%Oksl$E^j1k3#4x1QgzwJ}m7QTbmj!gq;pJ#6EH1+TYl$>pN zXcQpGW#Tyc^Tfs5KQ#qiuFE+X%hd-z8b|ed=*Y8c! z@dkWc%lml4L?zn<#LuTVPZNDTXhHiIIQVo#L;8#!-ug!PkEfMC#qp4y^#3Lef!W|3#2@ccU}5g&Pg?o8_3<$rDmcb6Lu zQt$rFSAr~UIZTKKQ!@JA(^L8s3_?tPjJ0ixJ-a`#JXw#wbW)e@wSNT%b%r@YlYBIU znvET7wHeWJRf$wK9LLnti>@7nPx|Ytkd6A8{lKNl^EfM4lqAkxIhv%$R}VBqcz)9I zg-4XjKaPp>*AD2Jp#Q|PY2fR+;<&(Uort23I(LV<3S~r({+*o0dtZWS;!7GYuz}P= zMm{Q?QYew)ebx>KF)niAF48UT(`_9tIWz4x$SW5wYDM?2W@H7&wUf? zukXq%JmjvcB;Me&P&|WO$V?OUQFyTtYmh_Bof2O>C0tuQJNtb;mys2+;IT5s374E zxKykB=lBFg;5z_3f&cvgW64KR-+Lb2Z;BrsH4t;`&A3lu_$6YA%LH~D(e{;`10~rt zH)nPcgRx@*~^!Utz`-jPu2-gs(_z(6T zoCc3o=YmNXG#!9#NpJLWq<5rvXxfeSb;x|IHc4l2Y?s;|>a<7y%u$Sd9ltGI=pV^4gSnr$35@ll=FuFI60}=uS7C-5PtI5KR?f4!aG0f zznve~qgW2xnXKUHI31eYS?6YJ_U#?()YK9tkyXWrO{=nYWxU}&c!_d)vChg{Tu3nC zb~24#*ytYQX;0*nbDp42?Pet&rK6|kn(vH@KTbl~4v;wkvd0y;L@{WqjG;8#o zl7{&9kjC}Iv(CXbIzPOv$4X*(}_aSbt z!y*$Bbnvd|)%kqr*3oLm;`+dY%FyX;bSzKfzK%t2(t?{Lqu~#>vkV7dTuuem>oQY$1)}df-4PF2MwuQweG)b2$!SoktNdSQ~zJ* z=Ozt~Zt%$Zch;C&UgG0YeIIkmAJwE*uX8e80=#!IYSGwJzSE|Gx>+gLJzP!Spc2ieez)ob7C{9ZpzfAt;bRHvEJUX?hKD_wbJF0bp zPJwt_Pe%zlhX2ucN?0x^hUFP*2h(tIlTaXGTBgPTO(_+EBd2gqiu=bb?T9|;_7T5( z_SesF{}G1=RlWa42>)v<{x%)c|J7FHRK*be-UWbrtp95<{;&IeM0@@}?S4yW^$fFQ zGG9}fx!YmPSz}Na!^}#N68AY%ZXA6L_28&0|L{81#fw+wOFG&33tHvVY9EZ%v*?aE zh;qnTs>RShlfZ?PT2ixN15ty02(GX>icDNlaqw!68Qe!+Zx%`Ki93q|Lj!_7X z<+%abn2Q+|xfR8yU74cN$YiOK3Oi7U%k)w1T*sc&BY2lIdj?-dGgT6os}&p7+WA|L zX4X-|#25#~>G*uEUAx+K&?7rnQiiB{kCv)n+rndXvvP?sHNp8R*${;)`LeTg=d8WA zHfh!7JSBsDQG-pq7MPQRg047NGN#1DF~2;v+A!mNMna-N!c~z17P{$7 zk&Sv$=;6Sd3g_jWi-JX14=2Pext;Aw%sJAa*B72QCpWXK z)C%m)+5GHP--Lyt>2AzF4!%6;t})U(#yt_g8XH2JXR#g9N`ymxA!KGR3>qFT`3}q# zC#ltWmj%Yu@W5F^+6rUOk?i_%GTpt7!`Q(@^6~9#C6qU~VTL}CpY7J8(QjzyTO7mN zM|G*oh51bxz;)+JXtfWwsHYW3GWn2Fa~Vq{q4T+Jt&gPBuEDoTy6GEvTO^p7rQ{9O z6_;Hn#tl!Aw?`H3MR7UE?Q%oQA-2K8wZTVJk^6V z&yPNu@T`>qt8$}fcBPb!O5|yDmDRlU(&0v$iHD^)?$5i(pL-Xl%?Ck_5tZjjXoNk0 z%=BJTt57m;BE)=u!*W6qCCu~nBu0L=YC=2il0Cbefzc*wpJ?@rI3G~4~e{kEr}l(U*^v@dLygSwH1CC)gl~LKqhw?cSM7zs7U7x z8k}1O29OyixN$t$ib97?f0iHl=n@| z;U)(oiijiywfnRjiD*M4E}V=hRMQ!!w3@565@8`5-HdN%F-(K1I@6M_u!$Z_pYVIq zmdD5$SMjFfAjfF^$*JNU{pORn3ts|N6PPJVbFd)9AxH8d8*Mp;1OjXF*DUm!u~SBN}(7%XQhe_0>jj0z|qQ}?qHV{%rwj- z7H3>H#?(7_{o8#7t9d>yaVLsVK2u~}#DPr!x{m5bDsONr5;7-;tPE9?Q2EiSGloxU zxF$IG#IXt-kNdv1Q2mfV0KrzpMdd!MKkT^I`!!^=jUuf} ze&rsfj1D{%K)Gfl6fAbf-bWr*X08sNMDg^)dJkm7R+Xr*Av?_TWbMOBov#lZGlmro znSDNIs?U9w7Q0Zv`yMx7_@JbDGBP&H>Eq?NM`~)^9pzPW<-XX95YL$?jS^}Ol+9St#RJrs0D02f`i5cWXP5Q+^^l~cXI?*S?cZ#9!2dV>(Pr&1gPH6xRfOxBF zT(epzrraO9iO5?!yaF9zP&==#$#hCX?ksM6uXf|e*-Q5KK3@c&{5}Cz85Xwr6~m0)x>YWKgPan_aA4E)?jPdQCn#Yj#d+_WR#eS{R+j7vhB@ z=qU@qUOa4*_(^CuF&F*uxV+AZ(K>Wfwwq zVjG!Oj+k~}FWj&k_3uJ7unHf7W1;93% z;)C&oKg?EzGejAtnSaBC=OJ(Sq+nL){MxQN>j!Z;rNb3rb|Lm}*mL1&ack*md7`Mc z?FBigsm13DCoX2lD}tExt&gR&d7rPB?)2wEy1k5MRd3n`c!mG1SiLJ#~4ur zMOQ~l?loRsNwTmN@}4(0`l4Avwp_2x!%%MYea%uVhpZZ{j>Xpb?TivzyYZ;>KOlYt zsG(0rc+-+B5}qMFPJqq%2{x$8?~acM5T9^P#IZ``eQRDdsdL;^i$6*+olA%XyLkSJ zYHQTZX#wum#qii=E{+GO=vaE%1xJi&Zufx=B-7M#=;|q6`M6qeS)g9U$7O$WI7=94 zPf`#qd*od|U8m`BQ{4>tljePZsz=8HpAoj~e$QIbY~Ar;xs3QsB$7K&i6V8fAZzXQ}s>#lCq=R7?JkzFIiXvw-;f6JZyj z*7)&Mvok>tyM|FO7#_Q{?fh6WevIJAO&9n1XxUwO|8TporaB5bhS>Q?v^$fiYjOVw z!V$6EL`FNSWg8SOP4_iIjdOZbZqoB14DM71e{M~HYfNk*?VUwFjx`Zmlr7;(|MDn_ z5A(~q9bqogOu}RuHlDGSOZsrtqSxx_Tjxs!8XI03c(`cFzS0+WgmPkL90k+u(h<_q z((P{aCGG~6d^4e5<&J|*VZB#au0M)6lm8hPasXd~n`}jM9Ev?T0Vp1p2tMv{O{?#Y zZuhbMt>@r+H%O}QD&(Vhc99nbI{OU^XJ2(kW0O|Y4e31{Nh(~hxdSoYjXaAcT%V$fPI(&igsgwOL4V0Dgx zC;<47ZEUh)k5=C(I|zTb=qr|2b~Mhh=@0tSpfj-7J0W~%<6!+S+4xcOW^I%C=j>-w zTWWsn&*g;-%=H#E3Xg8pZ#;OiG|k<#^w#4QV_f99+Mp5YSAYa16%KmXIov`KPcF%H zig+uz7L^Q1Z5-*IWRJaMU=CIBCL0A4FZzv_Zyx-qc0Lq}QD0xmBsS_d`VxEJ>f- zyGPQYeq%a{S(JDM4`PmA(rtC8BvP+OPS@a)?b1>XGH1STEp0?h{jO`Lvqnqn6HIY(V;5@LsY~s@Gk)G)cY#{f}=K`OFXXl%YKmZ5UG2%;Zhr_S@XX|o5PaVF@) z7!@NJ(-O^vU){qy;8$wkfY_Y2OA0UQuKB&|O_Ps%M5%;c+JBynYRkZ*1#m$Y&g7veta4$uC{Y=V*J5qc8W4e#O`-afe03cM+XmE*oaosoB&sNMrT;j7FwtK{|%0glR*_D$zFkk+! z3k}ts{j;>G&mU3XoI4xmc6|d8DI$bpQq9GI*qtD$#Dn#8XFKDk@iH4Io`zx;!}YjqEgdUTXB|s*vY$F56u9hjmzpm4dL2G@rRkht2t0Bb zJ1}tXCBUruyG!c{%0X0g73E=5e&DSlYe(Mx5<3C@|)H~mABrTgXd8^m3=Lq_#!{c*yO+?Hp}GTvRDwBQH*C5dDxZ< zmuqV67Dp#fDQ%DcIFt=c7pVq{HC>5^-KgRozxPl(xK(z9w84T{I_)S~e< z$Nv@Bl$dzg)fg_1d~wf~!YWi|P#&;j`nDRq8z+=EXnRcQ7UDlPb!FJM*)85Y60%5? z;!&{OUk=8~*m)>uQB;_4b~5-?nkC?@nMZ>oG>E_ zUnMZ?ebv+YTI|yf|eClF6+Zx$8#H6lhA|ssWqyE=BQh-*SNJ~`jopy zu6OwLt)Z7Z22{%w-zx4G)-MfNPEYX+F8Y)(*a$`9T4GEg`c)4p63^dl+{4J->Z z4A5KX?u%;&u#EFGZ+>8dUWN0(@_~0;+b~J1P5L#)Au)BW_2526jt23@FL#DIt&`2y zRc!4SKV1GSk$t@kS7v-Y>OwMjuhCgPKSr1&@h1O8_TIaN>hp+&1C~vh!t&Hv*HZ7z zje+ltb=8CM%9BhAZH}d>Eljww;TYjZeCb&Bhf1m^cJQmprsgpI3LX54sXHu*^2MZN z2>i5=l6pRQn20siF;sD1=DRgj&_LqvzH*chfKaYuc=LQ%ywRG+&4w7pv}CrnZPui) zT?D6gLG9#**||Y$W6_(cz3DymZLgBkh;SfJ;sR-5%0ub*E>akSj?s&4^TT}2mjS!17F_%L+!Jh)#f@1vsX{+i$HThJFd zX9dWwA%t0aVS%r&YFLWuc<`E`{VL_^-Rn*D`_MmNm{bcFxZZ;4!lTwg#PxUL!%G=w zrUrx*tHT%{fICjL<~?hm9SyYs8+8|muOeU18mF`F)$Ai2Hjat}a#HZ{jVgT!AgjY# zIgbGW5;@4)i?@*xBx1|6?8Q=LWVlM~=S)k*wjcQV%!&Kwc8R8R)6($HC3KhSuGP>P z)0$cIc!zd!FTLPQLcQ>Q=@DR-xO)AOTOavJA>o zGM$edk4^fFW1b{cj=E_-8XQy35RzIEVt6Q|I~QuIcFOyRAWq2lt0@vEd>UZ)HqO7P znVR36p4H{**Lr;vw@mPy+~^uJUQDf!LBGJU-T@>(k)!7NT(#m^}^To zxd+yUU965#P5!#^BOh_kEl{EzcXW)ZEuJjx#O%s1-QQD+aj|@=&(>g)SC0cbO7K_d z#z|bV?rAsaGw}v*P7IuqUt$as;e@|E?Vr8HADu66mgV?pe+Y7NV{&gVHI=CXcYruK zac*_~`rOc&b}d58ru;%`?bT{7dHWt1ycd5tg3J!sDG4~)d<3Tu zcz1$4vbNrScwD`9i?c2w_aQfA`}k%+({-El7pwCXH`QHmftnzfP@5(s4h-ZbDIczV zZP{p<`$j&h9ah5J-ybPfM$0LDjgoR2F{py;$${b=1i@Fy9568IU`=>Yv$+`lt{DEo z0VOZuttTTUHf3}7!$ESo73aKdeC3o!Z|QvblaI<1g83(IJ-mkJ(3ox?rMg!OK5e>J zB8rGNXnMG|>nEm1OrFKvJS@J}%We#Sd6Y$i&BV(T;m0ou)1fWy6wmPCh?EEb3@?G! zKdtu%=W6BjPlA_LGuKTa5C={28%r+k>V#l9OJ#*rn~Xj(KE>6j(cdX*{_b)7CuFeV zp671ewYh~hhnvKg^05=n%6kIW6SSv4tq&lu6gkj?jn2!0mDt+f#bz(j#zI(>xPPIY z(C)0N>>R>4EGbNI$w2TPhY9RFF>j*0>Q=rFQ>u2k(0!M{&+TAOZ1skq zh*r#4)xtZ)y`8?d!Z#vu$yo6$e(?Ln?{XdZ(q2eUw$%J5Yxswq(!}VfqS&i+5yyl*pZXkcQSQ!Qy^jC)UW8%xJLb|f4BU1|~tO^io5eo5z+mCPyryqjp8=NLF`!2|i z5^p3RmV zlpNm|^-1QuP&HlbO;=L1jI2L9xzWN`ajId-@eZUQ zZpgv0vF^PvhPI)t<4eea__eU$u&AUW4*8yHnrkWHi;f?-|u( z*R738lVU+dL_~_BAYDN~I-*DiL5g&gCS7W1p{R&bq=R&%Luk@_u+bsXJ3)F4QUeKr zvl8`rpKqT%zOm2yKKsY{!x%aOx$iaCob#I3HRoF83aU&_Mw+>Fm6{H^>gi4fyw7W* zofNin^p|8e>InsJ0U1~|L~&u{;3IvWuGO8HH011JN|ePtA8e2g=QRl<>nEaotSQ=0 zJy{5_>?U0a=k3%x*v$Gp^`A)B$F$A9n-gd0McMXZCcYf3O>F`o5<;*9s|GL7f==R{ z+-L~{%fzvsIwiMd&zo3&H#TH!YJ%Pjrah+UWQrK|K^vj5QvB;J)KEU=C@pQO0Dq>9Pvm{JutQm+GUgj z%FpNvs+_smh-)H=?`?&XZ-H(1`j;z|{a`-HOYi@XF4xUe*J_}uam~u-XjNInk)_8? z_2$d4kmV=tgmcO+I>&$})-=1xbV;sc3T#b8aE8KYbA*@BHQGLR$l z_k3N$&g-D87x=`cWf{l6gZuV+a$9%>QUvH5ovKr}r+YY@FtfU2A`$8ly>m4TS@_Vu zh35#R8t&-7TZin{slP-G$>X|*+`nN_R(9qS$AYFB+_Lci$czT>IHp_ROlYV@qu%J= zl`XTQdEb|wa-724JZntWCbjF9=y&GkypQM!Hfkb`4K2L-@m7SSSeIes4H6^dLf?93 zhUUj-@wW=nm-^`>#aW`}Ng<|DR?>uDI73SjV<2(g;|>ql;w<6nWUI0rbm3!0cIIBM zE!D1flmxPkbvyE3jN^N7xb>qe-FJTV7a6iYa?ZCeI>eHWq)aq3K9w(`(09#o^|2zC z(Yk0!{L%qX23rw)IMmMUU{a0KGcFW_ABrfvz;#nht&c>{g=qvTBh-!LSn)xkTzN0_ zEj>8qZ_Dzl1Y|h4J9X}Ct2ME!=(8}z+P*_|g)dAgNmn52I}D?`>82SxZQC+d4JHXg zF=rR|?Zw)RZ-)(if>-;#SPT%u?`{7AUVN-KV*OJ~OD8nlW37$%xwDe;p;-J7dvh!1 z!MO8_eccS#>sEsuKGzd_NG#hvLMdZ^RIqy~2Ffug1bAj~FI1C9Ms;^AE?BfJ(O8Zf zW=jP><0>0J)Uzr6C@WPIG0<1#xzj@^AAjQ4(a<-t`5rPsO5%Cru1jsj=%eKq9s}#? z6W#+|=y@*O)&-n19lp^G=rDATZ`YR2cF@exJR52>tC{=6h_4+-k?-XuJ(>?7uYLH| zV!y#6lm1mZ0$=TY18PT#VJd(k>JG(xkbGhzgE@=r+0z2!@6-e%N(F<*jB>o5BQW>) zmkI+=7OxLGd5>EJ{O~_tLdPAWgK}@ESVA*m(8L+!hin+m)s*6=v{bEy#qW_UPj+yx zMm#*5U237r?vcvapKAExxXr+arcr2dOp80E?ix<<51hG zpQeBSh=q>}JZQ3wxiuLU@{x@bj4u6}bLsp`C*wV_NH#?sR_c?@7n!8raewnY#=nv&F zI-XmU%6K>ueu~V(=aIVNg&jljh4C6CjVkPX+WnMOL(Rj!!nCEvD)DMVx3w+ar?-;j z3HsYNLiIi!odj+&d`yA2fKN4G$D&8qV2>}06k_qxLXX5J3B#Q9N_oXVqR`$K`y=hE zmqJ!eOtdee%KaoHeGA5SkZ$K_7O-`%4wFAok3JR2d57V(r(sOIhdH%2y=BkNB*6oc zBdV_$@;*6d>3(1`yq;g~v|Y}di7Rw{XDEI*{y*6tbXKP@v-6LIF1HD)bc`5%1?A)D zJ9061^q3tq=&rcw$aUwGtzm7xhgs@YR%1x4$0#( z=4|8Aqj*|^+7f3G#)Bi>lollhYv-IgXoQDRqPQ8Uxzsr<_d>asJl8eb_uBHgbY~9t z4wOZrp4S}9n!~xyC0NWoLZho>?EAc|mAZstG!}9Ulq1yDRD^LRVy@?Y-GiJ*{2>~K zcCQIIU;OyS&iFV3rJQO8BU6gs$iXg(sm!m2K-A({${ zE2E#zb?I>T%&@U1#}zXtMW@^=aR?`FzRDyK(s)TSjFIYG^aMz-vxFADsO&$0p_)4! z?ry)d@8+fd4&Gc8H3zpZO{VEIY2ysY8avTt2TA^ogPl~@!yICPG_TR3qudj;){;p( zMt37#4mGH4xZw>w>O3jp_%qrv&Ze$^ARudHtE&OCa8jR_fo^sCH#=*>b4Ibo6URG{ zdADjgn;q+VWDI4Zw-fEj=tBih*L}Mq=?rNW@8|!f=81x4QB)A z#x`0G?*C*o68R8x-$qTJzwvM+ncI+Qe%;|0_*Elt+K=ADWRCfVaGMOd;s$c7kwguu z7Nae`ZGbSDFpQPj`MfYl-5(XSQ&gH{D)`)F1IMdak6e=&wt99!&6pO_|joE=G9`$-DE+cXm zZ?x&(QQZWBx}cucO~Z?KQ|oPeAL#PiLzjI(&#Lq;fWvqJz2Q*#c?E?TrT_21tDjo1WxHt11W(r+MfgpUtQgKd<`1Y#)nk~h?4(j+ADm!4gas162?EI zl79!W1=-|-#1KvsKJ5>F;}}J*nIH9|TXKbw6=g>o%g(}`Olu+=g=D1=+!OTqA;cll zg%)@3DcWZDdseAck3FPs=TVqzapjA!h5XvOA*N_!r0%RB0craAHAvr>TXs9@Cf#xv z+c7LYi<2rFYGuz>n1(C-CFR^m%RzCmX~ux*k5mb2MhU*cBhy2yKVtUCwH$sctbR+G zqwj}$)VnsZFZI5@H-Cl3Gsp2x9Z19;2ND|_F}-1vQN7cZrH!X2qF56lOD7I^hJs!| z3PjTe6y52R7E{S}s*5tmhs$DzEI-djbb3K>(1&dB#F|v*z&d~3C)IG`-AZaBKCMe} zx1doL-iL9vSexRd$DX1uMbvQaco_9_aM@`>MGJZwarJ}bH5Yty#IGKInAAA~Zerbg zO}uBFjXwcow58BIVY;;M0eFin`H_RzE2@lYkq*p>KysFDrkLf5)kk@>c#(B+Z3I7R zHegi8S3DhfsxO1~4|ZrW_uupW*o%$Qw-wP=o!}1kHL?+7fB#8iAV_}5zsnZP+&vc+ zQVZGMFUS}={KAyLjWK;HVGL>Lc|c_p!&hOGQkjnx{gRJrMiy=-O3NJ}Z!?X==Qrc= z|CBP|PhA^+emweFlpS_7cvQGI|D)}l!){nmP`ISAOXTA4GiC~j*u!;dK7voV5|_aX zZ+xTLKTL)(|9IB|X~)=EGBVpX57B06I&M9lV4k?5@8)@QICbYH4gO`g+3-$#A!Ekn ztaQ@{=?8QrD>>@$vfc6u)V!L0nWzf%N6V{>oa;ZnTMaE7^PrmF#MBR4OZnpTFRaJv z8uRW5tL(eqVIbAJSahWOU82(nFXe-Vn1oYVb$s8qZZnmvq-AU5TdSK^o$Ql@i6*wL zvrFCtCOP%nBrl~X46YLEs-(3UR%C)Elgb>4?N16$yi4_KNhZhFed-F&-$)s98p7kn zu$LD4@5uQd%5w>E=!|(~?Pqw=m@O=m6y4m|H_ft0RMf~c@M(K=t$QHOWwQXNS550r zih6@Aw(B&vXc0X771Qe!Ya23vSBdt89Ie9**=M~s$>5HwY_lBEN&qs?cwT#W_;jO) zPfYEyGC{bWJQaBDhx{m&A-?t--b>2+!TT&gfMI?;ddo67`CegVOnKcan;cZJ!*HSA zlAyZHVl3@)OfUC|>PlxT*}DsfR8M31S%An6K-{U0j*K+*kkm#f|vd+|k2S z13Q1(_{z${n!H_HvA&GGPbXG98(RIivUvOG%}75Bp{JgrnpB2IXMNj+hU3u}tiW}& zzPa1ID6h`FGW(YN-o?R7&GtH8eavF!yE+{zzv(~wi=$%8{4|Pzo)5CE}w4;*%sJK4YQ9NC*!3HPE?u5ptVrLtjm;r@WXDW zqb@Y8c&=6bR;Jcaqc-JYoW&cL`WP^V<*8_-Vt2 zTxMl%Xxp@6;_F-TGh){A`=i(GAQL4fIh_n2dtJMslIx3GZ}O!h`@0bnH?oR3=3T|v z!#?tAil&xVuZwaR^&Ihe=BOn_T%QqY%?=Q|emy|Zjn5YJ@P~5{8oJ;9B|T?Sir-F8 zTv{$i(d);pM#QJ`I10z$3i^L?S{*&SH|CBn9S;ZWIV(JOj%0WN98Y8c&47s)Zb}91bx>bkcugMl-x>g!Gj$ogHRSY-McHP+Ga@?oX zc;w=8iZP3EbSJ&KdU^)ygrSjCV3-6abBOr#fE<3RhQboxQ}AUZ!R8)I9hKPwqB{FT zE(g^bEPTy-nNzc7C0OvC$3CQH>F83h(Xj1~-YxT}0}+elf#}NOUUeX#=eO^-)TL%7qZUwSfiuI}&PW?->LDIhp7dVTL;M z@o*0(lP$O;4s`rW@BKd9;YfYEwbUnEP2NNhzi6fp#r8h9LcQGS}rXE15?G zkGL+At-cM6%FM2ut{&;g_mdD63XE+PiZ~Z@B5}=P1oEuq*8W;>meceUM~6rXiH_FL zx%r%g;S0nb-9vEhAd zg!3mCpc(nMwyQ=@)o8<#7M(+dpQDi8_nWEXQ!mYr#?of)`xYtLtuSChi!kdaFtCi+ zGA|#>hysZQMQ{V{IK8Esv=iBnUv~;?kjdQAsnQQHJo56zb6Wm9^TwxRYimn(MS4Sx zghjKA3MxM>ucTZejDuN|t!h6R3cGFHd&|Re_(|VOZEhOY3?vV$V@PqNCeysHyEQaE-F7aUUClFF zI(+IpRvRe=dShuH6^+wCKld~#JgIKHGqd(m-(%og^%25NTEE)j#V*bdyX6ak!&5ED zR^7N(#YEKVOttfic~D!q>(i3Il%csgN-?qKGZZ?%<`aKlP|k)FW>5Tsb`%vron~8X zastPr88_n8JxUoB8cZ>72uYvP0rejVhP_H@arWA(;yR|2M z7!3PZ)Ki;&dqu0oInJ+AH<_P>7c~9)T_vOVZ2 z&XQyY9cGIDs=}7Zo|;|-GGWX$QMu6P20vL#IaBa5l>$^-m~# z8QqA5Rju5TWOBXVcsRHDC_dyCAu0agJQGyoS*?d(e;p#?n)k~V=7#<61EuQ4EMKhLw%qRfhHWuBIqR_3m9;Ckaw<7-xYY5dHT-c* zkKi`KbJMbKI~Z&{b#gbVRdq}p+*c|zif0Cl>Q=%Y^q!dMGQ!mC0wB30viJ>CR~{oo z6ZTA)>KUB$d|^0p^R+*vUIaoi$h-3m=%n&LNucJb|0W(jAA=QUKHDb-)I5&_In!=9 z?h*84-5u-6?Z>8h534fQ#?`z+D;S5EV>}la*2>34o83~po^RF#-}j9SPnp^EEN;-@ zwXI4I^719j0#Xd0cx7EQIo-ID>A6L<%g#KtSgOEa)>eIr2=xt&$ALT|O^TT?l4n}d zy%uhG&@JJ4@Z$D*Mc2AG8=mI+!FDDe7+l0zg2K$wzmwMlq92#1u6b#@S{uw1ETX6u z7a|PV@f~hX?kyYrxbjhV4|lULki(j(5na(USemZa{0)2tCs(P4o#)#>K`P8C5mVW!qkPl)*TTkDGHcLg?q+HpsW19X0{Vudcp`l7GND;u1C z+j;gnd-)3&G)%A0sW)Fu0wS=JW3(t~IqWo2X;z%+F3wy2IDokK$HxcRvSV(>4-+Oh zmL!KajfHEOuFUcJgoT8F8OP`t1{AoUvuvF)9fiYBS&L*=RLVT|LE9Il{*oL!bqeZL z_18*2RxW34RNxyMAYZ9caob|j`sl(1t=WwAI!oM@`0YU%(~I~+8l_(osA%$`6T@oB zs?nK!3e8st|6fVBNa!;wt1kR9g!7G=?fV4Xv!UHXRgEh6pYteqU6J?z(V;`FTzVNo zJ_0>kp>m-Rydr z?fEKhHYnUJ-lv!NDJSa>Z!uFjv_zkdh-|l{bbq705;DL?(pKez4PUZu#hlD@u3w} zJ;YWhf_bI(rGQWs+WH;F$5?6)hR6uq_P|3)`m3(`d=$n~5IO5HXI8Zj22L&Em9es` zs#L=nrq1&44~F7j+8qo1jsh($J$F2WI)m22P$E~P8gA-iWjmZLM%@Bd6x&sfhVB&J z8|#&5tagxT#W~HpM&i$o0Fy#s4*3LLcqANl3RJz{93DX-hL=6`l0jUFikPay`1TF3 z%b`PJ3EfY)~e&FdW&`)95L!U|lFqraW#wWCw$7BkyiXH!=>aWr#n z!3{Bo=T7}VkMxa&WsGTsgHQU$XqeV#4Dsa z@$3hGQ)*1FkuA>oo_agMgl7id$L4zjgfHOvki^!&DGbIj%f$$ zEtBnGtPd&W9>40EUi%>Sk~An<0Iq(}+FhQb?#`4z?51{VXY6kW!JensKHRf-i%9Jd zifnx94thwU3!5gFme@41U*o7k0a>t$A#2+Gdik^-ZOJ--ZEV)CcCVr&`wJSr1{whK zVor*1eD3j$@6mA%2?^RB^<9?p-5YrW#G4z0WT~9jr=B$LM8uMDK^xkCKUx{-yf`Xk0 z4F7_LV#mPn$UEmQ+UCA-(x*M$FgVws6!x)@f>&+I?)vM2(4)hrzb*|qQ{TiYaiRk{ zL8a(Yj<(!Uy=p(~sm4s;X zYIb=e*^%bCg-W}vpgZdEyTC6^F*k1l*QKj%)4w(uz5TnjIos6V1$4|)&$?p_aL(CU zeO0Z?@nerYBp!;-bDd4|Go;KYWg(b3#*u~sY+b7I)kkVYR(jn+v$e>ojtZU*E)J6+ z9oVu|J!~z`8$S-z+LBxt{mg6nh@qU*1XA1XRTo$s*M6$qol()4(Flob>nJcmuB!JK zk_*OYXa#!)^hb67Aj!seB^co>LQ$ivVTziZvjVf3eujLU#gp|fh8lkfCOEuRpP`?V zbY+!W#;s((R9EW?cOK=8AxL2l30=(yaOm$JPiN7YOmaHP*;R8JEjtWX_Oy92s=73u z@1>+=<)VLT?1q5kzEY)C*m13$UY+Ectu7!(-%sfe$B>UpNum>xiBBIv@MoHUv(~g( zV6)4ZN2gZY3OFmqz>L_FhU_p?0Ig%-OM;!{fnPoHwc(`+MCIbRDbKPWv1$ zq#6#~O}w$ty>~QS0vf5IOf6+obpxAdhp~gp`xRp=L zCM+drcX_N)hAaB0RriDUZ?fD7yK)LB3C%HkJ95{dBbMolQiVnqLs^Ic=H!Z>Siw!i zCbTlFt29VS3M@l-4}lF3`HKy3KXI^-kyx(PF;g!lq33C|zw;8_zHfH4U)5qCo2V`9 z;EDPP$auyHishQHiOR-ld%T#e(!<>4;q9BQ!~836Pmz`C6*18=W=#xdN@y6=)#RD9 z)dRI9=l=Yg@JDip*0&@%v!Eqa#POy&uTjaN_jPxkT|&1Qg|;+yMZc&pL8++lJeL;N zSNR%^TTW~uvfb84c4MxpgEm}6#iP;JOskGi!AvqwN^TpZcfGUQ)K%YLfkYSnWB9R-V(bZwthE~@}wBrvWEX4K3`&a z$5EP`fZUnRa6|!=P(}*I6Xa77Vxvr`7nK^Cr`J9va8eB?PUy>7m*+q863^kv7(2dk z@irIU(dsHq>cz!N1Lev)5(C{XM`d^=;bhcHG?@S)<7K{05C&Uk`>VBEboqT6xcgA= zT{iQ7H;h^4!_CARrFoZdJzYlFC1IWrW_~0LG5j5#w6-6M<1@l{dN!<3+i=w|M}RpB z4z9=iDfmtvX_hniqV5*cAO6p;O1cBP(k|H5?ghD|_nXqcwS9cjrF6RP@bPzbwS2Gq zM-io~CTvV% zrWm;^4n21`9*P$)y{~gdy`e4jL7j#~rBawxA0u0(SH1k5h``8`-)}bq&aX`53#TxU zj8j_TH9^F41!{96+txE;l8Zu6{jK1jtFJ%q@8i#`SGAr#I@%+z&WOuk)e3TX`0!(z z-&IadMfLOQw!>${9>aHE1T~})2bp|&psp!h;&hMkTSc{ zxUnu&f44(`)Mx*icZ z7eYsAbizBrWV55JE2gu7_KU4%u8}q`v!|PMs63Gy?QIQzR%958A%T6zo}(pYs2|;5 zGG^V|;NGmLE9w=xI^oD-Yj_&p%^q+)O~9kdDDhaG)+>JC(^GtBmdzEBdbNBJm=qdY z>};^U!kBX+so3m9l?lcvUc{oY${;-Ssg>;9=0f+b{u85y@0g`8NX1Eb{I<}-?bzM8 z^3xvlJIvRcCxs_^Xn))@3c+`!NfV#AsiQXaDPslv10+8D4+Wkk1BtxAxWs!+w8Rlk zTyuNgWTgcQmK>Y>bUaPH0G4v2b1SFx#uGIQq9|w9E0yM@ zr$WB*yo9dK2ERx~NRqBnfV*HbwDw$G=a)%`PxxN0CMKr1&+&aJU#NGp{3k3OX3+fg zihvn(e*Q-BRi^{8uGv>cJfBmF8Mvv(-prYRoBd)|G{Ys;Tt1dBDsK$BmfJ`x(gLkOWr}3zp zCCgFGn~&je^xW*5$)fVa0Ix(a}Qf%f4)`ElIG=C$jf2a>x_DUzQ^L*Tw~tBBu4T zQy7YCaOo#x!!hGKU--heBR}Yv=(s+ZysfaiHeM{0ap+Rp+d7%VB2)=ZC(`2VZ_}eOhC1YY#+2c>+)4>Rdcb1fHN>st=yAD*I>-i*m9^ zIyGHuoAAA9E0O%NB0)QX3GKQG!*uCyU2|^cbT+QQ&}N}olX350FQ{tB_026uk}CLj zDny8OM)2h+OE_PXK{A~3U#>$@{m8GTR;(q!=t{UUaR9vub!BeqKv5c3#*nRntmk2I zRD~{l^Icb+d1rDIZueV$IMwkF#H%Yu?U8TQ4o$o>tPg$T?qsj1H_L>|k!KAVuJ)j3 zACnIcR3NGk-K|C&Qyhdl*TR_I^Cs$8lCN5l6RVy$V*=4DH(6Wi1h=}-b-phWaurs!`_lP@lP< zYYdRqc4J}px_Bf0F83ATo8I;drx2GOxF5)vUM@_1PPA*n)ya2co~-OKXK5p#XTB5j zI8?2GwL)JzDj^i6__lZHEP|_b=c?|N9|Jzjwc0Us>;4 zSKmyh3GIQ|gHMcCwW3m`46kiT9tk{ckf-gxY}oKhzton}L2E^)Ev3IM>#~>8qjg7G zgQMwv-BODuu^-db5_*u2X5VDwyNAJ$9!7AB7ge2v`y~9}$DCb#sTKcj4q0Fh{Rc06 zSiVyk9&%h^5p%eZd6LJ<7p6air4pLReyb z{f1zo=BZ~7cs}o;(()=B6$F`xc1OGN713T|$G49X&%)*BsbM6$Wh6v7WE3T0JbER& zE`$(g(K$lF^01ZogZAliBW;zzN%+c&Ut4lTN{+4BP*$IoU6yRl@f5==zK3_*{0}5O ztXgM!m~P1X85#-OtVw5a`!F_hnL81U4R+q5RSJHA_TN~iV}U<8~B0FPh_6s$h+2pc^3}S z5R(oe#mk*ZHSLyadtL3#!mK+!v)kcY3W$;Au~!c!w*v|TJ5rnrr4aQWl5b1a6hW$} zQsK_nxKc<9(YoYH{#M6)!7c5_sVfpi7(3j99CxT}LDeII=*SCV7;5^{2b+J{jsu^C zvI0{R_gA@3rta?k}bGSL^|FVS@J>TX;eP7LS_?J4qOf7 zv@uK8ds;jwLXJNDQJ0NZZFetJ53xTF*fk{ z4cl||W+`6-o<8Kkw^8p_CvoCca8_RggWL|)VjswBve3k1%%oKMAO=1;F=k_IS)}SW zk-|{{;N)pyYuBdQLhJkWrBok%?OyR8>egPjHLRGP5~)hQ=9-{Gt*^#74rkYythI{V zofUohQj+!sS>1qs>^i-J%Ca|xdS$+9IbGBeOuL%@*oM2R4i(X?iZX-Ea zF^sH*h-c>?3LE%I47Xz)km3rvv#Zq4a?-?t7+@b^Z@o(3)T*!NPmHQ{%N_HdrNhTV z+P_)x1@w#B&xETd8jX0%q;$`3%nh@KiO6)<>}0+uJ}|doA#=_a8STs@yqd7-Wp^2(MO6^5LvPy>ZRHC-V1Fm~F*+PQIA9 zR}K9{y!t4xE5clg+;~TFysA-^ez%PK$iuL@!z1h)YZKEdTrKx>-p^X9k6U-sN`SY~ zRhBgD5&*?oyau@YW}4+;p0-DgOKbe7j&{@!kEp}w)l*{MAk#)50$I5}AhGgNok?@x zI$qa=&hUwmMHZfP%Wc9^T{4ksICAW*{}SPY8?F`7Z=RHza4^n&;mIX5^QN@aT;<&= z5{I-79mH4~_}1UHGe*?vH;>=ab+a8|EI~r#dY;P8B@ES);pVPu`WYq*m}sgYEzA}M zC-^bDZ9dqmwC!fm(Lv3~SaVj?rc2rtsn=2mKjD zIRuZjIr|5K$ec*>YH8CrdY|xtYwxl;U#SneySUW8$6}m4!SFeC96K9@$@Zaq`;9TA@e(nK|Kti^}mkeC_R zQ1PsU4z48(*?7>QZ$YTyr=)=Sk914$jL&Zj>6b|Q9#&^{d+`Ku8^^<8fP5+7^8@xi z2fWjVeSu6GuF zR)k+2FEm>{n6A++T@DNVa2mU&cAEs=ZF7G0_^?(KBd@@PAaQ}`A6mhcz#U!?l?cW} z^!<9g{|dg#=cbcg&1m{68+z|&imB=W_cOl+iy6+|E#G9~+Ra}GAs_Gy zV*UDI4a{Qq@&~(aLKaXRg1(>jw*m)B_q>jpd>RG?Gsn^%d8-Lm>F^ETh)PMO1@>>* zm`H4+l+fQpZQ2J?l?>UdtJESVKIyC3aUaY(#NWa|;LuN0*0SxpP(7vncJultE<0?S zKUdgZpE|j5;n_`(ebqSg153 zOB1T)9{$qQpEMvix7_{f{#3B*hA3|8K)N1&KT8^^omN7!N>vp4k-`+Kq}S5_Py+3< zjw&`s?>1BxdClk_B3J2`&FWw!?2&hKy-rt1B^>km_*6C%(Pydnpu&l$%U<}x)@3i` zZS^O+&r<=`=~K#Xn?tz8^v)@fy4W1hG%FtY+EET8PLn2G0MAtZ?9*r;9)a?`zhwz{ z-4Ieuii{R0BOVcaoei>%`AU6+4qY9+uwVG?nV9DHV|7%9uz{|D_RU6SL51Yac0?(t zMZ6(iRRbO^=x4lZwZdrBmMtlElu7VbqrE#i&EbIeeWQVoJEU=zOjo(It<5#7?^&Y2fUAg^%A*fiD9xkXC8`15vkpKCBqa!4y|A=CEeWa&WiFN7u{;iaB zbWU=y@n*_bJU%&-g$G2VxBmh}_0rYFlP7>zQBZQpKGyjMyW8(Ao@$Qg|49%5QA zlQWwRr!VDoE<;MHeZnxGZS=zG!0EIG)^&tKR#9}#)aaP36t{i+$Y*B$G3OmkMi11# zZuDweBkmlSkd)b`_Ww%vM%K@eBiXf&(y5=)^(k>S|8-`Gy--_0ZO5uDjz=G(L1L}Z z%Q3TDi$)(;8OUGBi>5f9(m1+Lui|G|Vf?{UcM01c?}ONo7&831;}pL?>8*Ba5%HL8 zIQZg~%A>?RTuM?zIsiR~ni=Z~!PUMEYZS zdheFMs*kZVbGB!HVE^n=26?s>bWXLO?v^TWGeg5VBiM4U-Os$cbswKjdd5q+#@iY+ zc6o1UkWIk)JHk&x^_8+lXZXl)NgL2CkE9VsK8yb8@YA{B9;3SVEkQgshUTm#}=ynxSJ zGX{qYPf6PRm>5%p180}m<)x81T%L~=_WNFz-BA%qB~o-(=ri7+CikQ*H|_1f#pu6k z^fS6stwMrx^B zmA~l!aGwNmD%Uc9dBpmzlx*;Y2o%~aJV7P@LV4bs1z$;*>;m2Qv>#I{UL+r@)4P6v zt6L)*ruo!lo&M;ZP*c&Uwr*@6mYHswKjOj*m#a;U&MWq;C(h5xlAHeM?4?zduW8T9 z(yTe>bAwC?;2n%6An= zzazHEv1}9B-@YQETX$85*W?sC@oN4FkPnk?+xeOePv+jN)HA;|zObN1*H(`>I;kgH zI6ArTmiNlb^}A|8B?6EA3_ay- z)E$EI)v5=^B-RMStT%EAMaL=PRqeCwARGCmNPn5JRv6M(?S);IH38^tHy)1eWvvPMV&%uEI5P{B^FYmA@jwy{%Y6I|%+nis~gBlj+E;s%&{(o3! ze55Mj`m}w{s&}UGU3#NO<4s9U;mw^DXY>7WS9M_r+%Dqbsl57cVWxDiL@zt}UcLBg z)%y(z8D$BKZD8x*xSh@M@}g2cQ&f^Iw=zw&d{+MIeLsn*bRkGC9lfNdN$u{77@@=* zQDP!s1`_${+Ix8ak|?_O^&0@Am;ARQt>zPR23B-}=C{KaR>vxgnyV;w6*uN z-KM}Ps&2H)IzJLU3vKRC!0@0KdGbhhb5llGz0umL=I4j*yYNl+ZQcDJ>yLOPMAH;Y zj$zf=2P4XkCr#-9x+_OISoI<}R=Io{LDyLYneL zepWNX(|ACjO1Y>!SvYhR9PRr7e}3-biQA$dE0a8+b#M4Qm#f|zJAW!S=ii!;T1~l7 ze%NCv82Jc`@ukry@hIP!4Pk9>%TjaeX{<619hkQCE&o1x5?@Xp^Y;Z*$d=CEvl0Y= zPRK&_cP`RM1(sMMx4G=WVX{A;VcbO--*RYB_Az%G zj?-<38NtUW4Nboq&!E%dRsUj=r#t~hEMmSY`bPTmiLatfC?eRyhJqxCpZD|h_`Z~J z<)>N?xWW%J)?CM)je$k8{fNup>{`l)kY~aNk=K;;A1}zGewx;;X)r9IP z1a#`{^~c5uu6HyPFaB_xKs3Xv+S45lW9OWQ#2Uj*LGH+0;bB(-EE-Ae7Irpkb7q)~ z2z#RROXzVo6#HYN`K=EE=3Sq;#k%CHWqIUIzrV(rGlP8#unMj!fW*t$Ty0l!q?AoA z_R@y~!3U~uj^?Mt&e^tTup}JAzPch1`b|Z+r8JSlnTG)6qxV;uLa&B<6Q>-*j=zoP zha8+!=Q=+KcsFYy7$;}7L8)!+dErS+r^NGzK^Hizl?P9^7TI~3zuXX%Jvyk+6)JA8 zq&<|sc9-^H-s7^Q=wfZXb}dd*Qtc3}<uug0JZf!mcO$>V!e+U=dTI^WnQ3_@rV4;~@%pW7ysalgO?kuPx6L#Ake)N083 z1#D>NdG$wH{VyKr47yLkwC^LM8j!7@S7Sku^*&hG-k?#%&2pVbGvw-n!~105xV+qJ zAYrz@hF7-MN>#nl%PhGmWzT=36PpP%tv9NGJaMrPaE&G4h9^sXyu=jeYo!A_!XgF@ zF9oqT87CzIf z3Uy9vX|Xr?e45BnOujIHtGzSdUEEpq3g^~bBJC&fdf0%9i1a@2Kl8s5+MlX-3#@0d zaF*;}Pz~JAa0b*)uddqt{>U-nwky(p6NWRn_(Xv!x|Jb}$B_UgpDMVc6MEi+Pp|yV z_)l$W4!wfhsvNA_T3iGPn@7jSFxVi}u0?O9mk9w}^w0hvAVTotxe*{bx3bIsc5SjB z4gPT+W)jp}{O4FiNGVF{Y%T)^M2~2sD5yg<^5_+W6qF|E}dY(XM;0vkIw(4n7JET=|nq{=HQgqAFRK zHS0LUVJH6nB&jvBY$xL*Hks^?zd3ILu|ZOn)VZ21(g8jJZ}~})Ke6xkwk1uX0}EZx z!9zH8^w*hG5k zvjZ|k?1TKT0Z8yUX(u3#L5A)u;~D=to6f~kV5S#rYI_o{ZY}bVWe(Y3+MV6$pC7*P z?;cu};!=|*q4wh!?_|dT@%+&w9BU)&7A6Tegh&D~t5W`>oBS>?h!#Nf7jsYy{{2i~ zfhg`=X)>6glpX!^5z1eVc_;jw0@qBPm?z`}sfOuA@l6y7z(2tm#3 zKissW!xMrQ_|0mI7YF@i2XeuFmiyKWj&c02we|mXS4fvT?{p=WUE||Jyyg9lud?uO z2#N(IynmRn9Im~_2Pp7%_}$@m3TzsolH5x%V3Xty-{J5lZP(XRc{df4t}|SFf`%uNRQC|@N3PQDE*%;ERR(=M2(Ldu-8ieWzJ*7yk|i~iuB*B zVj$oW)FlB=(1aueo@T5mJ>Sm1^g#)_J+Au}!m9##XKcZlnD{Nr_uX-}&9OLND@cXs z7sYg$b*}lsA<4iX3}LBl{%1?w(>+jS=@2(gLY}lp{I3{AQGFj6SLygP(th#(wa0?< zE6J<{VVn}EHIo!MzX>5`_G@qaFc6o|;q;>oc0*+~w?*=b>c!lHc11M%2~4y;>wgv_ z{!66f&L0ItkbPnZ%{kEG;)mNlY*E}gEl0)YdcNCa1(X^(Z2z0zH?g(OOSJo$-WKn` z?fj?+5IZ0yd27f|SFfP(AtkdOZ20ISu7|nDfC3*A*KBwdeD*rYhJ!Gk zdSrqf6LWTjQ4xx<-)Sg`_KjOn5%DUxN_5M1V&q4fAU^G|5g%X~X)JhOfQLQ*m}Rs^ z6ftl@S?$-MIC#HXAx)<&QDeTPc6jilVg+@bUNrO9^NcmBoSvU&dr64~@1k&Rin3&V zxKs{M=^}4YA^MIlLA((TA%@0j-apk=z(e&AsuBU6s(eC|z#puli^nRSz2F@Azs#sg39yP{q}%V090SZ99;@8? zKe5Wq=gNEzcEqbjR6>`}0rj5GxG=;chEcbUvG!6I%Nb^{7#oh%u3Gs0Y4AUaOyEgW zZ}k2Z3BxagXP*#!_uoe&AOzF!SzA@2x7dSit2!ttsuiY!=nxS-RnV-7>$L900|h}p zd&7m|3xn`b_T_)*A^$b%YmE{AAYSGC3X`OH-B~ih%g!>zMzph@n|!mDz3C7&l?PfI z;u7tzb70c(K;g!J1{6ez+#sR!4d8lK2Cyb}Ch7x~ehz*(3D$zdHJZWeAPPm%7;v7k z^Mig4k1v&UE>l|Pbgaw5e>3C%#y3w~`s+(ggqwWklok|6g06=;FarYO;=O$Ey+(0= z_{cxOp&PXLKqjLv;N_y<(DbFy`%i?yzZu#6Mf7qs@TM|HHm$R7xW8|1AMagi7qXDGd5q4FL&WgYU*Gql zi;uX(OYjY%ULbXP{nHSpg6DgYrL)ygL%c_dU=wP;t|geJQdi`YIl(bl@cTN9@{dUV z+kgLk==Sr~n*k2ew{}hHT1p`CUZ9CNXMjgn#~e3lJ0%-X+Qw|W6`4~K7Z4ImLvB7VPi5xR< zqa)-%UWVJX0Fh&W!|W91S{VbALIt|z#R*?2@E?Aqm(-}P6PhWS@}9*ta=#9g456el zfB66chK2B7)krJ222e$nLo@wb0$elA#_y~5|J1(9mIh5>P8+7l=1Z#;y?dFW&4~{M z%%SrZU1fQ%rOk+t$FK#|7g~Y9*%|c*`P*-yc}rOZ$9eGZaVF!ReI7sd`vot))}(3+ z{dho!xp61``2$tp&JFzkR6GgvRJrGfE!gJy!J82=Z(0tRdI-CVw}Zz;a2 zOOy4kz3P!5+ST&4jpj?i-g+mY)~$rE@gSi6>Q}etA5`;ivKG>hnuMDsF$E7wZ90tN zx5v4E?|S0Zh|t`KJ>s6@!$X$#5qR&E1l~zt-v7ZBKpyi7?qiTgO}>1^A-I;C4IJDM z@mB9G55DwRsbws6SRgv=;(7I92&7OxrGMqvT{nu=3$hxANZG&Q zWZEy?ltsXDH~&#A_p47TU9X?`1Mw=|I5a<+Q%B}Bb9dBu1Gzb%C)AB!(wu!$1s1?P z%P2309SJBgIst!x{5) z_v0vXk6}lzCpP%PSzVO|5PC6PNSypbbtb&4B>wXu^A%No;3)G>z@4-Hcc=^w9$Si( z$Q>mDmW~efD@p@w^h@FXLp%JNg1{8(Lzoq9YUGhcp~XO^VVK1x$U|>0Dw(CaX))k? zR1UTlERKRsM-Lv=$kxR?hYs4m3L-oAGeqZZOS)m$g-9dLuGL^YD=6yKw|Tb`T#C(w z+?k{h>C2>^QlldVDid31eb+1FNK0B1;{%+c)#S%U<;(o6Ps6@c(5{;8!1e1ENNrl| zuvc+jvo)4rOYcl-0yUub%8LEpIVYAWKk2&*%euOW*Lv4UPP;_05o}I%V`?AW=+|{$ z99JEiWj>iMadssDHg%t&aZPL@#bHd(lS9kKSjZu3%hUSGc|Q%33O7 zm%L0q^RTX_xY0MqI~j`B$FA!moRtTtE2VljAiP? z^p4VNPzi(*K!OkgA$&9XDsg}BI&1HD|2Wq<*ZSi%1!nS;QSNb%F`lR1q}1fO9|>JD4$ci-&1N>BdmU!%RI;<3rAsDTLvNjoze!3k!=*=YjO;mA5uTL~$}= zDogkaQ9gUifWMq&xB3#XdvFscVPyDW*>Gn?PaG51MU@@(VtZhB%E#H_fY6@z4QW3XDz{2Gs~8Fx6M-{efRggp z`xg)t!2T9r68>Vp-6F4xS})w9ne_yJA9Z?13*4x(sh2tfdv=g!aJvjWDvvx|Le#D= zs7n`I@1Nh^us5_HUvkVK@`9(rmt?K%2<22$zC(70_zicHGl;Af&_ zXE4xQVB2}NQ?|jEk)y4=74N=ClD8JQY%ZGE;=CY&W;MneR+9Z31)B3@Lv36mv zqTnjuf%!nkq#Jlf-QgbB^%oQy)XL#$kLIl)w0dh zpHa_*2$mu*@5v;vBTkweU>6tnkz@JZdA&xHoFBksuP2g<-b;9we%x{K(`0;p1*Cs$ z&f5OC3G=V-PCxh^4J7Q`)2mgy_o%}E(NiXXD&*E%5@kqDO25d(|NhfgfcTXY!Wf%*O)0eJ8_w87ykRhKh6le?re^J_uqfzSD+2Np|Aj;XmIUibST+g?(OQo zdkUxkQcYnrGK^#w%Ks0RP#X+nyca^GJbqjoPByGxwEb`9_ircHqy)TfG*Zd?j}|W6 z3%~?Aj$M@&|6t1j+Fl5FeY&AX>7Om!08kQ@>}Aoz`r}@(|L1xB&2ODge55YW2TUF{ zLkA11z)6^C6m=MX>el2KU5sv|l=&U@qJJ#ff5v=G@RD)=8&r#>oD zzL$PKE=eKNJIF~jvN)8@KiWbq`gC|&BH8H7zx;+{`xnf;pBJ`u#t+(kb#fE>ag)n0 zkYZN+u3@PhYRvG17v2% zY2X{sfU5Pj`U|_fu7P_mub%!0AypJ2Q<~QI=1BI{B&{*xf3x;xvNS23@H+56z<+bA}&ueouMWH>H$OmP1GIE9*nkCI8< zhd5qm-+w6&!_>g10ayCYpZWbFFv3m165yua>%0C1V^^xjuc5~pjmZVPSxngoos(+DC$c|pJqG`kp_*pldr zG;!7^>j`K`yUQADO3$CN^|?(yTJ8wXxm&BPTq}A_;^7_kQx^3h0LgiCZ3dh*x&z6~Fovm3OOL*^tr?#&58~c9oZaJ$O`u>otB@Hu)yDGX|@Q;*i*?+Q`_6q31& zNF)bTz4n9Fy7(84kC_fhaWGTSGOP*LJyjy=>8{xdTz#s^i8Pf3w!nzqTMLW!^L^ zlnb|4F#Ly|YX?^4jhxhSbTzpt!5_Q_H1rPTJd<4;P~%?Mz}UMixxq-4WZl?yo}}|Q zT5xF@b?La6^j@nm<$CS2D)UAVcJ`aT z7R=pmbVptxbovPYiLDFSC!4p_;Tzb{VrZ zM!oFJzg&K3$~X8SGI7Q}%WIt(`#qeoyJ#yuEz{I*MlMZ)S?vA=<6e32*jd*^OH0eP z4a^d@QyA3wBBGI0zm%WHH<01{)W~m1AyYrUE?U?WNwBh9Ux*RGguuGwv!wNnAxV!g zPF46)jjSB`1;%K_K)QDEo$c9`!>cwG`>Lh65?xonHe1!xmaAs#si-}pW^$SPXx?}) z$_G1tC+_l^;R?gj_mw7k&C@gE?;!kIOEyePz8j-g;v{@umdpLFs{Aji6mb2-N65!y zUfeYh8G^Xs7I2>uBS}}I7-~P>)1@FwXsdO>eLnR5ZG;V=Fi{9%M|8UmA{_2V{1k?3 zVR49Xtx9%W))|ck?HYacAhfr_#j(t-uebA-{W2x1`|>7lV(WLp;>r3(XOBMz2&BjO!)!A|(bVAfz4;Z~Bgn^kKMYmw6jAJ^RviiarA z@ag7Uuwj8D)4!Tf&5bYRvQY^o?BGNo-UA6m&eK$mJCde;twv5dRvv63Eqz-M;^^@E zd(AIr*Bxq?s20u1C=q6o&z?SQ<~UGLSo4i=X-#6bJApGy)*Vr4{QkxXh{cFZ%P`j< zwO8nt5=!gaQ>JnDpz*KNmeY@L3T8|IFq9u~uSNiI7f`W?qC_G9pg zeRU~EtJdT%>gBw)f#CpyQt5(cs13>Y8oOABi1y?sRDW7}ESghI=!n>ld7C)Mw<%n^ z`FyqWc9p4ja)S*#EGojW*5S0tL1$=ol?&^P4n> zS|q27F1SXBISz?@K0TagXkqQ^!x)#NU72;%&$Z-zIwM_%U)$ko&?-t18yumb4KG)& z6CTXclPnjWcMZTGi^nJQ{LVrwDIqhdgv5_889HZ%#*Y+uJCFRH=|d8D3^VlKDXP=& zXs#6&b+j3faqQSU0)IUYX1@AePcTfPT+)7w*L9Y7VL`B+9{h0EaYVZJH3yRS!MhB3 zzC?LK$R{Mn252`2)*Q@uqd%>?+mx+KwT1m4Z9z5zQqB>QWkw+25gl2B{O^scqf#}O zs%EWn&MjSD@v*FlF&LGV*hU**7xP_1O!@ArrEB8feh2N|J^y5-s@YQw=b3^KL`4R# z$qgJ?uzRzyJh|j{ndJypbb;bIj>S1!mP5?SkwN*^so)d%djWEH-wLo=4+knCLSzJ$802H?glG6?y5aefrI z9$Gr$%=zdf`Cq($L*Y*MWhmDWg~PACCq~eQgnMyXcn}{4hcO3BdOhwdpA@+x?IGVO zt90!-eLO^~;Z9n;ZZ^sLyk_S$pZ*q9fupf7Woe{**VYQKg1Es_wN}tXix9 zR|;~gK0yPIhPxG+eetn1N4d&El7Sjci%*?C{lVV2n9eN|tK{tuBcDVz8a-;4kdSRT zqT2Lo$r8eMR;w$MKz{enWSR;AZSonZS z_RByA=VkM+JlKs4NkyP3sT^nzekBkznG@pZZ#?6ZSCtlqLZM#crWbQgH~J8Q2syzk z4WS4ob6g-><*@gz2Q3j^`R(1j*H<@x0yaW@b@^p7 zk5@v9MO!b_v*oqwq(RT+(|Ui(=__tkYT-i;CyeVGPpXtXoWyu6EaunR=bfx`MIWpD zUX(mrh<;vo9>58rPiEx2lctj}*-!W`fy^gPxhkzbrWyeezqwx2%__O`WPL$ef-z5adO$R{Fs;+W+p%3`R5SZ$Z_l{54&ft7R#`yERrZagbi_Eso{Y@f|Ntb@pWopvf*P>3;ErEV*g}7N9=C z5z!kJ$X3@np8K^qxU&LWC)T0V^^9w9f7HU-cO7BXHEY3;Tv;i^cAT;@PwzxN@fMnd ziVTQ5IMZ6G_b?MrMP{*E7U4XM#y)kJ@$%s|&cTRbQx>bCo%P{L?6(wL+&JMkP0e9u z3WU>MMwA!}&{>P;RdVGUbDZFt2^OJirxb~|v)^RjOyBZELci0NZYR&-x!!V6Z1kn7 zb*fl^aP|_Yjp6>fNC1)|WuZHz%1+xggwJK@1M8g~>@-l(Yumz}QzzHY(l8vyou%8* z;i-3U(_6|gjlP{xvp9+l+6vZwcfH+z){PQl+`G(8=};s7nP4WfygS9QmL30?)#495 zBMZqtOOQeUhegpHpqTNu%?|jy{~%of`er#z)d4Z$Kse)jr8T`-BAZ;#xC3>F|r?lU^{${Q;RqE5g&Tnb-giRhSSyJCBapr+~jx7r4T80?;E`@GQK*x<+Ue4LhZUhUOI!pjtK_Oo(Ac;PA6D-99*XX z!hSq%JHFxRst>P|BWX0UkxMoD5@mmS$;y3!OLr6{9uib;Z`M4@6v2Lcg2zX;CVMK( z_9QLI@eC*v`B0{DfxBD$I#iQxt{GXgP_Xsv=MP5npZGzEtP2Oyi!Xe#v)=Wo8hzIY_5_3zsEwi4RfYKFQ4F)IW$NR!@^6=h>0W z%P7*Rs6VCYcr7VLdVOHT7a>1iVFCW8S?`OGzBxG?A~|vIY3&+9wO#rxUGulJ69W0R z2fFB3rFkbq%J1aDSK*ox+*$J{`Ra*nDY(uWQ?w^|aRM(;xfJ`s8q&}dHnXJF*u4}q zoz6@@e6Pst!-~r;CBC(m$M)gvOe5W~a*%FS;wRmO!EDFmw{>g_4M;Sk_ZXljdWVa? zRPP5o*tkS%J0~5;s;taHCA4OBmLI#8$WEJLWZPE}CoA)vVePepf2~3BX>6B{a2)z5 z`)a)U*1c=t$wfbIuKg`~WGF_Kq%=Vu`rpZj+kk;nyudL9nZ`r;Fu<#9Qe^24gl6_|5cq^_4+O;(j6w-n(JgLnyBhuD=0SAPM+I(>nxuY`#7FFcFXZZ`2)8SeDGixs5`nbkZzkb{SPvM44zczlGM_0TNCkn8G!|@ zH?0TGC@h)RLCxqVuS@u#B4|-DFeKnn2v|18jjHQ;bHh_PlAW7rTj_!F%W^6+>PR2o zXrS@pwqD+rp+n`m*GeIP&@V9X>q&G#T-(%nYY+(Nu64grLiR7P_MP{;l=FssG8QhWe)`vlRc3Fj$zZd(;4NclsnJMw7^G*cG~24a-mYtDI!xv&n-9k>f6Osf=?f{!bTCImlJ(PJ znV0cK*$Nd~n^Ys6G5o^pHIm4Q{Jgd56Zdw%1g?JgRRtH^MlPXLmDjdh}0#Z@^U}qYPg!S@-z$`-L^&BdQ))#>o7NS+^*wjXlRR<9C zt8G-LpXeC~wjX04GI%Gt@ahv~7h5<#vAgAk?6=RxBD-NzHA!_OQgR$3*M$W_80=~* z`)eBdWVfM85m7Fk5*a)608M$?wm0=S#6L2 z^s{%zmxkLUYxAcuo^XRVhSYSBfJF+kctjw(m$quGfYsT~D0yG+jyz+Toul%9;YtrU zbrBA+EaDiq7qF;j`2y1F*ZfL#RtPX5W2+-6y=M?UCF_~V zHV+wBV_}v;k<}q@s?cl8bKnb{DeuznYeo-_7bo{7R)X56!LoA~J0?Bk%<}Y>N8fS} zp*=wTI!CuejR0eCysPs5r*|U6{(N;9L3z+Oqt_4jPIYLXcLg!?$gUwLG73AxpP{q6 zv$_3>O;lT+755g+K~6pWGLt_H<}eK@KeVIq@#>!aJaGKd!Nem4+cxK9OZZw{)^dj- zFK2syhJUm9ua`YOPc@X-M$OFV!`EcqpSJQ@iaD{H*Rbp{H+3ypcAIcK?(+1w*`b{c z$bJk|M)28WwMRP_%pKOy7MY!!8#s8^0U`y3bvt2z4eA$oq*#bIJ^T$LJ^nE>Xtf!B!@7@ zH@T9!I_v$luK*C#eK2sPp_;j3EMHH=Z}Zdk{xR*=<(g>|srjZ205smz6_bawP%pqc zj4 zK2%Cm6-Hd4RWAkVL!&^F$ge)#I^Bd!Xyz?PR|)y7%qeK;NqkeUytw{roZK6|H-3cG zw|xL-xUy{MhydWoWOZZFiCouOFO`mdMH5mK<7L>@-7UT))*zEvPBp^0o`3BbHTEMV z_~n+Dz|0~Z0OI;4tQ(Jqgu%(Kn|~xk$yBrAD#?noY~5rhx8fW{3P7yPr@z|AuQ=n* zxoOnZ=9v+8_l`3_)Vl2MQbAmW>*VG~zC1$cjtv&pdgIOnQ$sJ1I1NYLm9h>SS;Ds{ zYv={R?kGf?sEcUAOm6wqDTcNo{2V|IXQ6zzX7e{q08JbOPv&TpP`JB=AQ{(V)_SVz ztM8|drMD~w3-h#%r##QTSFbV?sa%s;q^8`_)<%oBhVADAPJjF*c?zV=oY&ifBcGk&yD_tVvDo0)$&9)~fpOL%cKiRJtabD@w8y}6) z0`poM*bun1&$BZ&*J`u~VYxP&Q-IDGY}dV2cduEs$a8gndwcs4L(`q}eFa~%irf}Y zj~2PPj#M3#yVE|Wqcgo2E#a`KqL)K3%$7QjJ9U zb6PpR^<4exviZ>*f}i=Ko$hYj|D}+JExgjzCOzkf(w?M`~Ez8#mb#&^kr> zv~Oi|q_eqy?b1b{xL%GcFezcL%IZI3BU>LJx&5DY`;WZ+M^Aq`i~s1t|Ic(`*Jh<| zIyGZ?H_WdB#$Rhx|M=k5h+C)BFtu;|l}{D;pAR4LKEU$${&l5;VuzWzByL5R9ap<` z`{8Ct;f0rHUi!a#!J#A#gW{J25R(`acblH&t~e?5@}fMOmxoPH;#wSf+ktpx+Zk|6 zr>Qd@mKb^(Mw1nmDqN-hSZ@+u=-R*MlFrd&V6yw`*<5%9{-tX)3-j+#u9ovqOx=6$ z@TszWw3uf}AdkJ;h~uID9d**Qm@k$e_{je9xBL6?v;Pe2KP&b(sOf(sV_p|%G2(VZ z(NjrodC^0{G5V%(4z*Z9WYmqtp+g<>OVSRpvbagZ)chPjLWi=aVivZ69bgn1Rbq%G}ALe+b*QQ zS$NhL#lv@dUJI?%h~5kZ5iO;@eoL|1-dJ)`%At^7zq#D_JT&_Bp^|6@erOD1i#)$! z`NJ58NudZimKCnx)4ONwUa!QsYNI=bR^yI=U?t%h%;7gsmsyXJujS7=u?EeMYff;vivP2;Z!(DSH= z_KvO-M5k%jdXA(YA+AjGG(X7TGwucPyZgAh_%t)>TCAUHec;v-4TGtz%?esB4&`DS z>88?Yb9K`5qGMxQhQTrsUqaRn;2HR1Tj9jy*M306BNO6-74B_M_FfibbYHn(Wo2cH zW{H-Fjx<*7i(l`j@~p`;$c~N9HC?&{fM~OvpU+^=cik1Pxwt$+68VprfW^wnoX72XFrl!=|bH>H;>#Y zuIYaS)d~|Ta$kxsam{qx!fi*L_5JRQ9b2SF<%r5Ov#c-*-sROdnUs;wQA4k#+QA&O z%N=Srz0=!wYH?E$5n*yC?L4Pq6FY1t5FQ^D`?9POJf~}1?-2jVE1RGTEX zP$ReIb&>h*VUGt@@*A(|-%l-hv=5v5U8JN!#;?YF1wAyW1lA@t=B$8bH+_ zFXJN_nW2%|b5LSXD?FxlV@xHPWhYwQ(+cQ3tafLeX<6Lv=+NbVZ<9~s4%{cD@Gf9d zYF3tO(pDNSW%pcmdxBSL2G#$*S7YZZ*m~Fb16|SEhqWd5Kk*^o5I+szEawV7v}dwa z?3&y5r`Z!Oese|l#BQb1TK2YhH|M)CFRj%2z46%?!9=O&n@s5HY0Z|#uALOwmc_>gj>&hgl-Rh-Ff-+jT# z&H_rq3fDc=(Gu}EijPYOQME#)zq4FEgB6uziRowqE45EtM%g$>@Wn*HUq)71wDnFr ztl{I75k6Pk8wDT5d+Mq{e{E1umNei`G9FkMlY5yq4S%YQiJCKHWw{FS`}wvP-ijCO z&MzNA6)4idTG{-rb0g#~ElO3ED)R-h5 zS+=A%^DLYZ!9e4r*L^ED;uGF0ZWp7TF~VaIA%2&o0@oTK?|gFami99bCQVx?hrztu z7S+&stY5)K9-+p+T#&HZ>CRdl@?ww9)Y{G$wF;k^u8L4XSKGNMzBXY~DbYMmkf=*r zIW@LZzvtCrTkos0*xbhiY>eo%G**-|&?k$d+Gn7v@;RZa{XAT?O_*KWHSZkAwdhjj z$(RuLsD;$if2^F40;RV*OAncJ9*_P@I)k_|!zOSJu-p!;#CyVRcoG$t-qHaE35MI{ zZ7YsdHYEhnbGMpGhfFf~JwL{D5n=mDd+U3x@@!86CoPLaWcu#byHl;qQ|amuHIp+k z(!^7b(Cz4?UOCe;$0?!VDW^(1PS1A%zMC)1SEWwcZB$g08+m>RO>ct1U_8fK{bWKm zdwPMwtMh)o3(k=5y+R0&awDFpo;X$zY;!oJQ3c5NZl{VD3}2&mo9QY0expZOG(}lL6MKTRBw(`)?)F zBqkyW7Cwq@@5T7kWO0IiN#=Fkqz+~8+ogrT`LKH{h~fs;)WPHxnSbM8r-<=g|HwQ7 zmcon8yXt`OgkZn+HzPVtI)0(j3wM~DTIH~#Ht;5vDxF>~6A;L{W*~o9p=W+Ve!~)L zAJn)y)hjQj$1FZ32wDCb?4?#tie*l@jMIvEZ*-v`Xr-?cwK%$0Q{}EX7JD7TdPl$Oyt$2(lCy~<+%CRAa$3;#T)jHMr zxI&o$#9)+wEbeY!{dTZT*k_GH{{n(Tyb&HuXpp3HQNq$_ixMjTuU|Y+Dxy&piHEca zGvnUqaUtGZNQnz1ymQ7CKQI`jx@d?%_MS=}&U#qlt=X`TTyQU41281#y*Gz{|1dyR z=qFy-Uanp_2M+PN+R@R0YFJL{n>(~KPa%a0Z?wXwS)fv#E&6)6kg~E5-=F z>d~2?2LaeNim^rg1<1U%6JpasHw7J6kXjz0j{;uMQx^A0@Bx^r=P{V zy^ZbkzxU}JwwO2PXlkS>)v4fP5pX4QE8ZA_$eTNlUJwMRv#(`Z>Qgiom7{D7%osYj z5Y!y8wOpXi{WO_IfVm7;H7Do!kAJ(UKv@k|5ZWV?6%?*2SeeR|P`RfgFQ>Gs+3rw@ ze2H4JBHcKlcWS z=Qsw1u>cW;?Pw8zD)Tc%HUqrBwkGoA$bj#fmU=xia<&o%azTm&&%v+vAecfMk{55E zyA1A};;A_WN}e|zWgZEB`4Tm#j|>Iv5JD+@tpNWN2@CUl6g`k>tWOln@T;NN8L`j| zhn7zDk{`apNy#iSLL*c1F#QMm)!y%A-X zS9=?3P;>>Fr<*MSyd0_-T|2rG81OXa>RHGub4+Y=2@CIk@s(?RG+Riz!z}dR!u2>F+h0M))9kfAsSrNNLc34)Y&ZV zmAja)?jr?NJ5(-rdzn@20Ri7nrItYo)o<*i1-lY8CET%-bm zHbuo1wWI5VZkSW@b!9pZP|EbGKqx^r=oOm+RJnE9;4B_;l1a{=@G7*m_zHIIf{52X zJQHh}owFsn!-8-VD=L3}cz=6uOzUP8%Esq<#3>T$x@j)(M`NTE33Z7wG45Gqj__iV zU&{a{%lhcPUo7Bin)@<7IjpkWML?6(L&VKNMA(;hmPhfGDsq>(iNua}=w)RQ8lftlrpdAiQWmy5&*&a=km6{A7dagM_^uM#_>5OS#>;d2NLC9=% z01Y+MtwlSF8{R9 zde1pJdA{!WC3eRF4^0813WcFOW6s>Wth@l;U}MCERst7gn0hL=AY>&VzF(Vbak3YJ z(%jG^8%yIB;H2Fixr?9S{TT^9v{j2dQp0R#6ro^EZ%d}Q zxT>3yFPWV6eusZvG94t@6E)6A{BD8GTnw6ygGxHo*yIG&KB#e)?T3I`7lwsnd~x>N z#WuaKKvr_&Mm;*Sbkg6<;&vN7imbbtD}^_n;>&uXW>?sG;l3L|veG`$V!@OBT(vzg z-E2MU1a9QvmBd8QZgq~ugyuJ{#Dq{hx~-S+YJ%?qNRoA}Hv*1ls?@y!b&#HgbAr}H zU9!Uo6OXz;D$rS*=-g{Xo!l0!&e&qk^oP2UrV>mtp;2&Fsb|KifS(@`FJ$|}&+`C& z9_nC3mS$BDBB9O*@dC69XN>d1W~vQ+A~EgZM1_{n!Y8H0B`jB>p2<>F6x{t@E=@T6 z-#y#YgeOUi-Qwm(+OvbA4-lAY0p%L?R*z0=a zxDCJ#H|7z=5R_lA?nATPDW=-|&6Aq>`OBm2;PL__z8(DUp6l@KnsECwqEV)2I<|&5 zK)z+>UFuMC-tqG102}M;q0~$!1gFwSY;2s2#~u!h=F!3eJc00?Wzi^r@auWJ;xRCs5S&ss z`EW?#?z7j@xIOmyc-hb4n5Umy_3}Bw=)6xJ#sh-;pD@*go;dW`6@pzma%Ob~_4mK{ zGGobj4!Tb*%5>k06Pj0{C%zsk5W^qzzjGx0*698Gr{&#(d&z%}TNe-BH_MMMbkW%x z)AJ)gh;D~F_p3U;s9l+C@Iv!|YU`?OctHQ|S=xq2CaG!*%?;XVk=Ew`{d$(j0~jfX zXv&=@c*eGprM+S-XodUYuHR1l+OD1&b0UPm^HzoOq(b+jgkpI z`;~p9@PGdcPpMI*7wq+YD8_yFp6Y~~KHa1c(122K=rvrtEiuOb#okqIZi3xZ%}Kf;R={#Pe%S>O!vCK5{S?uj(RR)BJYr=#7*W{6&<6hX9KsKz7sfqJ z2lh7DU*Q}nz3?K_wRdkk=6|IZ-1$!Ax5AwM{YL*G0VbO6zc%{Jms63f=Q(Y1L=zp7smoctElJi`m}p*nj)%>RtpuPd^82 zzhjQ;LF}*Rhtt#m4J?6V6yMumj$1DPyUciRVPa1b6h?lodSH@j)8Z9Rl-iR>iE^@1a)s1WZvtwyTRJrMl;8~a5233ze>*P}_#ajO zTUD3oovNI=iN$A&&Y%rC-R0w^PhdMwbQOF-+1%lfidBS4J!TS1#2lsO4AX8CrW1Q2 zbPn{Sp)KVUwv#eY3>eq*Dba@3K1{pjqEjzo0l80IDm)jV}iyL zj?%TI$YR&_iEN6EGMx>*W{XPqxBPY-D|+SZv%WSkMdrRhKpz=7Xv)d-(LV+dN<$_V zr9DBt^OsOqz1XO`AcjJb%Z!S>#Kv*qD_fITy?s(suG zpKOnW*Wx3RYDcAFqS*1Hf@U7;K156pSbrT*g{mQIli#r*kW@IV2dJT$TpTS~S&sTo zU9gGFZfe~;*|Y4_9!9{NE~xgB_e0r~D{Kw|8OXZIOnxi^{+rCc@)D!2u2jp7)hQQo zzJQN={nASVmF}Q>gHu;xjHdmGH`y5i_11vZv)2q8+jGkgPXIVSJzz3cj4~@q_{;vs zukJHrgAdEUv1%9QXnT=%4)NkM<=t4q{>NSB%*-RM^;sxIJYkge5`Hv+eRMP-Nk@1r+I2B2e{{igsStq+qMrx9=Juo? z?pgAPyb-fxkWRS&UCiK$h3(LqMDl$%Z5=60&S?WR)Adei$XbM7`^M6!agj;lbWmEL zSnRZrblZ9sY{kn|!f|9l`gs)K#1wA-lv|Y;&FhO;E~D$;C`?C?+-WK{3ccq=;*NU(ub(7p`0s()o~SZJ~jGFX05*7z=~dGSdVW5PiBQH1Hhg! z@(_+JZu0L2RCPR&t(5at<-v{n$Xh)uo-Y?Qgp#X`xQ}jqbW=ahgCl5g&iCzF4rd~v z1kt$dd2pV^Lm3EgDezaq^u`*#DefyFWg9)WHrDnf>29Ua+CZseTT5wzQL7WFzxIXd z4=c2h(HC^zxbErSDgsG5XxNjfHokRh7)SK}&Np#~o|mE5(aH}Z4CJbxy$R(`duKwA z&opG{y8*c2YuL{wdun&QzyRQf68clz5Ac=yt5Q~TURuZ7itJ26?p_9{a5Sq2`ji22 znh$qXzPGP_nLfM|xRTK2r_6GOU7Xmzc~a=_Y~xR(>1ZR)JklsB>82#__rEk0fD~oX zVnQ2e$~9wk5jp)-*C?xHma1Z`MACXKFHY$#%YSS&U>@=P>)a^=sVA5?75}q@7 zGjVKx%nc40)kaM4aEICsG6IWTx(l!yIO`;Nt7LUn(Qa?EWKJNJM<3b=bf}#9xuPLK z{E63q6+LTXoE;7q8g-iGLO3P5p@}? zJN68;J32kv3y6A*?L>utV@e?1dVn8rbp^>?z%fN1j-Y_O%-LEQwrNlIgP$;>xG4Im zKKbz%Y#-$HcyDoR<*T;cQ?wZTxHH%O!>|8ZOeTY8dH^B(OgY!;9JarKdF(xkk-0SX zSsR$f?tWCH>HUHBXZOv(+NTIK@_z2SN5cP^g?dciRvIvA03jH!wD;lppYVU(?Y&M} z%{-r0Bq*|#cC!f5s8>qkaz=^L``DeMfO+RU2&BYq%_K#neU(x zg>F4;d>uzIGA#KOP9q`eX704zO(Ezbx=1jK56tsFEp~8 zeOrBJe{+Ob0spnjM=1`xV)*IRMmE2nYJoX<88ybo5E>$fijU#vooVg#urz9Sd!WW3-lb!@(m0vR8-B1dqr}~DF$4&O*1t5i*e-n650M=OH~lb|$5TCrd2%O?@c7m% zhhx~_9{CqB?ZStfJ0t^vzJ-ZdNOa!2+gB_$ zfn62r3ZNKcd@e0UnqCUcD*jlX$ESN{nlv!c|INTO z8K!n^B5I@vQ3}-W0^~p~bntC$;x-Y7H#Yjh2bdxMZ>K2^+K zJmH;Y{nQo;-7CD1-7dUvfRGD@5(zG~@l+c|&3rgdK!4F{(2ZFY_YG_GZA~Zu`uryc z*YL$MAmhR8hb4=56Ch+hM1kWDVD+A?TS@?MQnTKXvRbEoJj{>V6|~dtp#?BtDtGE~ zjZA8mSPg0)8AIvc8v)8oO(hHno9a(t%Y7xW`wkE`KNqw9|EgGl4DCRQqnFN5{@7f00Wb5&qC^&#cPUbLdsl<>e(y1n|SS;_)EjUF{mJx1}WCX)Gn;P^m+I^ z;J2Q>9GLQ?@6f!#JqB>2!POHhm|DYLgV zpcoHZSVo1LT7C}@P=^Lw(yO{jA{Hh!TouV|)@^N)BeE@+IK4kcf#U_0d!YGhjj2_G z9Q)?UBiRA93qs9(3r*J8@nXXpOMUT>L5x_j-}=zmxYZ9n&di%Tgae-d7qJWAlvO{% z0N`xdS)hdn`iVG+~nQtj3MiQ0O(%fB67Dz+^is)h?Cj;3j)DO&+9?hLGuXH7cX1Vj0Q49dW1Mt8=rzC-`M> zQA|YtYRIBQd!R*KBm^$hHD5LGfsYo$?`Oc7>OG)QD=@w2?4cPk1Ip|0)STg<#qg=V z0@~-J{_-JGS7}QDA3`;7JpBUV?$-7uJ9+jl05CQv-Y( zeAzn;B-uCz?etxZ%_ciUnEGv8kl<&4w8g*xENm9=lSw&!fXOllpzq#WZ@Na9Zq8oK zy8*rW;5mp9|ICpM^%BMSJ`HCiR^ZLT(03Tn`gDzds`Uy$ie_klJ$1N|_5FLVpvABJ zjQKIJty<2RnMx0r#Rq1N6&`YxpTX&>&NmC~mY`rLZsi$vC|4b&KD$;h_H?3==uW_`2~FU~>F*_x*?u zMgFmc9i-aGM<#M=(Af$g7F`q3Uw5Itu|8WkrsQe9V;?`&o1{31&#&G(l&RNWyIOZo z5MSrFJHOl36*h~0RQZ5%8+$N&rb+EsY*{+{XQ|z$5A5v#vVw$WUjgFnm*;DMq~iEb zWhxicg|7pQH@NeAryD<;BB>Cx#o@Vmxzc5;3p1oM{lE>C8pSQsqxOr9MCw(%m zy>1|E+VEfIK|aB*RTlt*9LPpB{x9Nqjw=C1EF)@Jx-n|c-J;_Qc@nC3fY|bC%l9OK z@v?`mEV1rFb5t9Oy${c|y4fX^DVIrVPUHQiwoU4GA4KGYh{N-@-D;y@YB^dp);Sb) zWf;E!`M#~Dv{50j46)mLs$C@op(-=G!40Eww@H5PJsh))gMX2*B23f>pYA}NfP00i zu>vqFiz***XUad_nZ5duO1IAuG-a>9T6?emai3@fsQkC1714yFhd-zsL7^#y!_;ie zdMR^Hnl=E2#OWU!64V%ArE^?DL>*cJr#cVy`pnnd-a1kgFbUDd&-h6hs%F7F^W^NY zM)XoAlpQa1fmiJx3Q-*p-`<-q#*a0Tz^0dONtWY*LjzeOB7_ngn!bFRKH0btq0+^0f;GbHY03Vi-B&y|L zHd*6^6}{Hd*BVxaO}1~4XSbKu=`AU}0~f<$XH_suRj6(7j-j>Y&IdR3v?zN`2kvyj zm-0AGt}x%#F?IaIhAxzMIU^p}xV1ss+d4u_eIJVBh-gPxN7Hs%RZO_73Z-}CV@6<# znLno(`#iyM6qsUuBGYgLoJk60co09sZ`0z2vQa*U*SiYEcg%RO0`Xv_?oc)fw*K#? zNUe&?;ba7TL>9kZK^R?dB;x6Nov%t1$OP+Vmj;W;zj=$9^wJ;A9(3q)TB&NcUg28P zZ|n4^T`|x}qtLw<212+$8dgt_Y)x$OOuuR}!!re_=JF7ac$AK?n`4?si?7%$Ki;~- zqepF+oq5C^y_v%kOuc?Y2$fIJ|H4u$D5?@pL`jqE)5~B62Hgnd?IA{Vvft)|LZG6c z?(6XxTrJx?FJNvvq=~Kbh&C3*UExLi06mm>Bni?}xpZJOy?Z%RXZd@Tk6~qv=eSIf z*L+n{Pb%A+06EI2svID8PbwpzF!t%^lmpnc z09b1(;4}mDf`q@U*?3f-Y#gDMHjeI^Gbk0PJI#o(91t?E_3PFGxx>8DIL5solj6W* z2H@-NfUOnx1$;h0kSJ(nQx-x``etE}IRSQBnw0Bv3CPIxYza%&^y&b)#Rf>?+@sFb zKGYkzA`|=~I2)Bp)G0~^%Y7*is8yacevjj{x%q$Cd+(?yvt@l)K`{WLA|e?BAh8uB z+klA-Em^JXJqc2Ip@s1cYXJ_ z*7;`^Ome?_*RFc1>UnBg$yiMnh*sJg(+Ylt_EfoTDo&g{1d0pN-$2vW^oI-Tv|Y@I zh79AL@{RH0_%6J-ieg4y9VROU6yGD~(7cThSjkn$I}a$VMAn%GtlyL6@231;i#-NX z6YM>sT5aweV&~wo<~IESru7*fs1R?~Q51!@*4yO`f>=3YAU{<6<9h}o(|auFPyncf zin{6F#8O&FXedU%w`|$N z+PU1H*KH#l_oba8p`zmI%yRi$S#*rGwP=s~&N#2lxNrB`s)3rRPMdo@ARP->3|@e` zz|TBN1(5He;bd?id~(c#2TTRGelA9{{!Ec^^2r$W#pDTn0QpW*weJIvuiemY#r%l z^*))m&G@~rnC3eH;$UOVHLx4fgxzQ-SxKv_!;Mis_`uG!FuDc5=ekgb)c@>8OzD-k zZT^cA!}?>9S>IUCgk+ORHcXmPu0n{d^MZR%q6XVwy}RhAISR2W(ga%XZ%T~&YZuqY zk#H0Pev{-bEx?e?G#mP3)FMu=NrT|8W8T0d5pRMIl`*d$Y@QMZ9zE{nLH@@8Z(04# zTKf~q!)d^7Cb7{0;ck6%!mX0*pSV{iGxvavGoscVy{)w<6^hmBw-iiyT5m*L};1t=X*jn6G&3fRTB;xO-fI545my_L|p+ zhSkX)wayI9V0w#IA5I>mdds$sq0{~L!;x2}3MJ%5ub->+)9q2>C>0 z3NwexXJAv2H2z8JgKR*zLAlU zE53k|W>5(B3Z8qv6pUiNdvX!X53))Q^zMaV2W0?{v$aL=qS-~3@{k1nnPl>B1i~Jp z#PfJo@mgZs#el7=kIBRp5YG>dDKUapgGok>iQ(`Vf!H?7?da2) z`olE`aj2j=qY34qrTJAh;f=}B4^GaTnuhj4X}J$qZuy*U4%Jxor*n>@d)*cc0IFzG~K#`ts#V%-}T≠HT`!=e6O!4A7qMF+?d24N>^EB=7 zsnH$go@Tl*FdBE_?PpfGHPflv~AXAyNFTF_L%ADo!X zds1+7RDawYmw6}YV`6KUN-;%r`YYH-qlDr?>*N?qKuP?Na)Q^W)x>Dxqy-^d{m{t7 z3x#Y!H+qmVo+RX$C+3C?@mNI75jGTot?e-6Cil=y^oB^hjYoT`iWFkYs!ATQ$%J#? ziHWKZ=<@-kcbh>c*|IiUF`G^@r*l6pnfU09y7-UVqfeOhZ}d1DoiKUnK-(@J zwGl_gARr_>OS~%1W!78Z2W@im`4}szH61SS3W(jPkVPpe(sREFWxUy1cNT+Fan+uy z9|D-%`@OCEGKojKTsLRsHFK?>(K~&(JaT1Te5+A!{cG)NP?t;PS2>}dr1$?yA$$Ub z5n3=s=z5Uz9r)Wh-y|SQmC5BUxvt5lY8FSCyqVE3dv)234OPS~%GjF*Tx=?)07Vf_ z?BCl^81@<_*#j#7a)rgxDJh-z_|e)?@wKv zRsM?~>?RCBT1jBSugw4jNFY#xcq3a@vTqV^0-@7bE6Vu;?PozYud0pazL6{Ot20AV z4?nK4A}Xv#p_qk0SOY<*Bkb0Z7HZwD)X@A`bj};@B7b@I6^7TeYj`MuElwENK|z&YAc|7A$Q>c$IghH5EY9}!>jnx zo)lF{1c&UnKM;N7E2a8wsfIkE0|kD|u+V#~gi@IF-!6qofdcrz+-7L=xgv_mZ15IuXMHdlUJgGl!ie+5yiNPj9bID!fut6jwaHn{KyI0Q8|vCZWgX(Slce z$<&vx_HL}S!C6tdDr;@ziFhmrfRZfY@><#pr{;-w8Pb4apGXZ4>@A#;o#>`)FE;O6 zfSa()<>F}!o9((*0wKta;(oi;S?}7Wp&v3H*B{EBrcrNdhavtu2$${?xbD$_4{5x1O zU1%G59L8%pDZRci0|Ddbv?#qsk(731H1KvD9oX1)-jrRjK3OGOy?16~G2tvem_e8MQxmE4>)YR0w5zNiYD*y=j=i#= z8v8@S8a`wPa+nEq&Fm47!{q(#9LAPEN0f@u=Di25Z>VsIYi0SYiott9IhIC}m65bt zt$2^N-9mJi22D?_P zMA%~3z6Q)KCWT}U0+*G3itR)rm*w_AGAoR0^Pa)hvKQJd^s4$oU6z7_D{HZ7FGu?6 z>`cd~RWU+k`i6;~yXzQV9$HLctUfv`XdW3KZg1kvmV{gg@aCP)4oC#|GE(TC^;TkA zx6Xp$b)iIY3dwrdxSI&F~vJkjzSH?CP(G`_79M}=xE%aVnb+08Wb!TdsvJJK2g%(`#0 z4!JJcW!&3A1Fd0plNAR95FP(a-=@lOUYsiNYdw8WVj;bgdHUMZ^E%rm>K#-~Ftz57 zR%3<_dD7GZqL!;$R6x zoVl$OMvrE$*ZCfmDMFn$#Y$pB=9IcZKN}r$piQBhRe_Z)iTBuHe|EC$d~I;;a_*cU zGL8#pjXW+yW`)-cZF6v*&J0wO>uhSiWn$0)nHf{=mB{bu^ZBr30A}I@?ESxIU}# zBcASe=s&+JGIY0nA@4LlTi{Y&Y9S70z_pDHOOc=sF;#^V#EQ(vJ;@ES?TL#4xI(GOsEcC!1OH<>6^KDaaWuiP$m_JV%eXRYLb`O`)URHdZcan{9cMzp1Z5oKA*EGzL3%>vx+GOjNYYkpE@ z`>BKaSULr@W7L)6lt6sh*wl&L+RJZ?=vzJ%_eJ%XDbfN6o!rEmXGh51H9?49-tuZh zzs=zdkwpOUkMK?Vumhoq@;DUHgQk19-vE&(MR=&$tGwlzLUo{f$ST~&`iqKZpGP%8 zY2q8?4W32Q+YexmVrrsGXP^($qj}BcRnXouviNi)!NV;)ip9E)a>6ky^2{Z-=Bt{Q z+`PG>^0fsTqNdakrl6oRm)PO00HW!5t&sPW*sBz^ z(M!GldO(@3l~OPp)N6emqPkJ)7OLAr@gHcnTDSmL1*o^d(}qyY+c~JghG1L5&g1Ad z4#ZTQw=z%$mttQs=IJ*<2(XEfQ9$|Fqe103=M4x!hJDg2QMb|=R}*n0&j9ss%6c$z zVYEdB9h)4#voT{mh4(UV%5CxCiA9GGtyU>-FJh>zSK~K;yKoEsU6f^L>JO_NbaJb; zbHgmPi8gnKOM(_DLcgU(?ZJy78VxkS$@GL zqvW{Uwzl53(@o!=CrcYh@0#6CJh~C5&&l{X0PxR;-+Bv}A1)V>Jwi_b! z%nfkc9JYBvj1^#;pAb>R9Kjwcw%}pYc6~8(wWY~GJS(W_ym)9wE?efr1}N5)+gQY! z3+qPF7kJH>g;{Wh&u`=N=`GvEvPMDK4>PR4|LB^T9_&@7W|< z27G8k7X!Vm5A_Zg27}349aI=O3wN)@;6M^FQtX1aa~tz*Lx+IJ31kUk6{7Q{`Y{uCt7{u8ZHZ-HjUwThdk2?G-v zqTZuWJAz_XIwKB604Qt;$Cfo4gzGY7y93c$?m8mq#xXH@Svk69F#Mo(_H9d3DHF{~ z$rX#i>jL2Yks*Yr+;n~18Kgz$+{K z7{Gcrx@|NcoUiK&RxtpueC+p6qd{NxOWc&m-Y4dqz)g(~KSFi(arH5e9NKP^(9Re9 zw;e*>j+}!pfqp!-xb1TgYz1dT2%UMz;D%!7A_p5!MR6>)Hv!T4CJ8{WX9CcK+^sCM zm8OY0^crz_+Nsf_x0oAA>>GSp?cLplzt_D)i){$O$D1wYT%mD?|Ih#qBZsjVg@-g* zz+?w!H4S6WYv@>2X6Tel0%;M8Q@>F^<7f9*1uvmipf+}4P@?K0O6oB0DFl*>*O7PVJjJEEaam-su{pC#nCz|C@?P`|z=rU+XXcV#oV5A#G?>`vVO ztWUhTD|hdP>a@SqBT^v?u*-;JMgOBfKD1Ph%c#oPsw z4IfgPO(58sCbC;X07o>RPQzx|b>#}NNnG64-oI*z5IQ4?#gaY0P;gGg00zrJYGFZN zKo7eI`db$bmxa}%81f7U#lRity4qJ)Sh`i2yu+O@wsCfz_HJyus$%()W!H>?nI0rn zU5$sPQ^~k9bNnu{PlmR8z*b1(1(67CH$wxti9i#9f+(th6~=3(V$-EMtK-^YRcqc; z(RIVuHwn*N)fcd}%#@e`70sXVUPaW;i8G}2OeA_UE`^DLIUw~ZAYx946RXP!)teiI1NTNABY+Kwsh-{7}W)`JpP7EjNAp2>9l<>Z~-HZzUl4-w%to z6REoSN%leA5nTScq>NiZaCKTv#aHoEmkqdsw(?pZ2?GamTHS;VvuJ2$yHKO>F@VIr z5XnxNuaX|4C591dR`LU!0yu& zj`b>t3|QaSJpsr(aG{h~Lcuv>5u%kNyDgw6+qr1Y)>1MpZDeIiS$uZEh7~kG4l?%5 zG7Dor4ml1={b07-IZHZes^_4s0ny|Hnk(fChg8pPBM%t6Ezq|`$T>}dAo)qLk z!ZwD71alkYUA?FGb|k;GjTt&hFk%AZ58bX#HqxUw(~g@WbfRB-uLHlZB4}{m-Z+%6 z`2@3g`OdLMf|TR+29S8r%|>R8?s-u)0w^`hog#0aAUcda4q6Zcn(IF@o-i9SV6q_a z${rIkX6h;xU+)vdVG=BK8WQ5TIMPmoN=22OaaL4)B40GXhS0nq;P{0roEqXOtR=gau zI!c|?BslK|0rVH0hR&gPMYa{&kBmxSE6HUVG1W^1=|71}N2m`;yi%?$dX>;gyZoAv zPY7~a?y5*XHOss3yLzX^-kBj9OAgi892-mC^Rti{u%-Z2tM4)yGd>n* z)P|?@dKdCYAnrO9!W0pEe{KVbHPr9j*CEj1*@RVp*m|U=u^G*KS+ua326OmT{DAwR z-H!5YH&(*HGx;tnz>@y`rhTLYd3l9=3ScqY9|m>jIaSLgC$q6{ikTs!w?VqaD!RVv z)$Eas&Aw{!nb-`Fj+AeWRg}a0_-6!fv%dDwiG1mC@EkB=&1>nq?SA!#z3IbpsxdRiN$@J@D3C55kdlZs1V-*+jpYi?`nPR_m5uQz+G!c46mXSC10x zx@IG$$;|C<7PQQ(1*&FrmKneZUoU-dHoOYN4MgIHw_t!Bwxb#GO?Fp=Mj3Q8m!#?@ z`3bFB<+0@b{2$nxAU93PQWPJ^eF5)P6;Guvtn)$7u5Roa)9S~Qh6*2fl<1I^ z2OH6SjbkUq46Q{53j^bemCpJD{|329JxBA)#?t{|RPkkgW69T-2w&k4GBdBF<$SgM z+7WF)#*Z{z#mi#0%3&%|g5lS8T#d?1dkWMP=iKgeqD(<2$NVj3K_Ov)O6p-{d^@=7 z$;RJk_n!a`bA$6c0rtDuSnf5IUrp$L(rh9=1G8!U!3J`y50WD;=hMs|5(vvTzuID> z+p)hTxNFF1#76jAJDx)g@uB|!U%^KIX=jRy0U`-?XgFDYb6V087bsSWGdxt{2ds5X}E=oP#!fuN)69fj2 zr}>Tm(VtWWOdHKN`9O&;41*wepXiIQ|-3~$5Iz>fo zpkq9OXKF_0hZ{oY?MDO~1Oh$#n3#dfvZE_%E2UvVK;n_S1U7cW)rn>ORen?_Dx%7Q zrJ=l1D9^^kYTIhlX{BB!4}l>mKI;)<6IVTV)f?IeTQ9PvvBQ|;*)0~C-VpMacu%;B z>SKTJyTgWbGwnk`v7G?)2x6c-0@zD3NM5(+99SANbV&FY-PTc!*UquVty?cD^;gHq z6YP!1QwLr@q<;`jt7gzv_9^cko>eo~GILAFEO+aQO{8hx^?du`?Ly0m!;Yj8oq%Jh zdaD^#>_V+8BehFd(v_)2+iSUcM=8|%kB%8iDfXmYK$`N})3b?-S~sh+tQ!P>v56tE zhh`PEG*6&A$L!n9?h<0-4m2_>)*0h#`)@h1@9_1&Sv^pYD!j|GuoM2&OsmNsdAjSN?y=slzUzAAPib8Qz8AIm;s2}xP~85r z7r)c}{<9bVNB1IjGxmZ@q46oW?a0?uAOb^&AHM_;uKgnKn{z+^Ci7u}colR@r2nb-KD)M6r zX(KvC$eQQ*mNhHq3sT_lNQs_A-~0*Hj!V2g2{5DXk?Uipf0j8s$UuH+DB=C>)h>QJ zL2|0D74WrxDwqF@q&bT)73_B3(3GBy@_h0yB4nU{`Vd~QHbH%?+%GW;*N;br@PrY_Qb-<<97sJNo zN-nm0Y6WjS$@jg_iu1V+E#Ys12%#=MJcqOd@=uDczNL<}t;o1nE^bK}knp@JaEvtHm-_jSQbFaiqVWZ1#3V543qbd2<^efXNnW1lJTH42Zh zqbI&uzk?216(|DaM9GApp+>iriVW7ZWf%&VGy zJI+sa%Ep{*2_?Le8bO0szIJ({UIH2OApT|K7YNm<$^Z|K0nzk|<4w!d=AjC06vXAL z%lc9FgM4htCnS)T*~G!iqFEbCv~y*NbhYF?4OoRvbVJ+2N7O{R<^|?FxMTpl#_hsk$~5tH5_`0cyo4ERN8Zf4uuwnguBeo9~Umn(kW{K zuZgpmbY_J4$6ELlG?-Tijj5&H$qlLyHp=j25aeiuT7S5sG`=#$m(<94-s@KHtRHxg zl}6k?4LnwU(|)d>z3wFyC4^+cqW(KZTbZg_5!(irW=o;EBaEpU6XXjwkm1L z%L95hUf(6jD4a`UgZHEa(pSEw{eB*zxjJi`J0kCbyfpEr4t+e)!T0(~2P*m>1&BOm z_<2!wcdq&l(V1?)(B~h1*w*pK5=Dpg^w6iPLYT0aDrY?s`@YA6qhAhWVoli)={y zX7kwgu+%*2GxZ@!Ot`rFejrA0N=3*jnD{;svBxyS_1nh%$~Lyzpe~IXaQD={4kopG zy0J5026cj6U-Y?I!7Go>&Gr^`Ii=JC9!#?o6FM~9i^BhdW6#&KG1Ri-^{?YgdmWp| z@D;at1xfH9JL=^gPEI^ip1v>QXLOF#F!L2o4b|*Xc}U-Q-l>O{9-sFN(m)}Jqz`A- zIuen@L}EX09hDTj`q5kUz-LJROtn7#$)%ih2p>;G(aoB>m;rr!jhn$!Q5T3{Z>`&N zeTv_1EAu{&iW}g<4t|N^d-l_UEeMx7Y>?og9_qylA)?d{R;FArbyXvgeFt|}X%1oM zREXBGpviYIn1tLdND(N_)>@73?-&jv!}ASBa*En-o&R+Aitlu~`tqr%A&Zn0j)_lN z0zq#);Y|e|I^-$-8cB=bIhT+wDZf{8JXP{@<;CS$AFG@{ar6}0c&VnV>(I4NeA>Ca zu{_Vv^bVa*44xe6RcBeP%)4L5%dS@Dt17f2L#|o&CfvsD^mgGs1&ro?e#bPe0B{ ziRUj{p^oAE4@V9JARf+)hB0iEo{%*YCs}F7k!?QGb{|&%d$W-~^3;A2zg!Q&roU<%A_+RyGOs?haQPFqc0W99QzO`qL2rVN|gU}9|I z+iRXzw;J7OHj*pwai=cczHR=<)X(b%@p-DzeX z#1QsHr_$WCRlB#SRKcQwwtM#FLd%Mw^GaxMAuL?R!Y^2`zA7s~Wf9t!9{({4pVc9z zTkN!6dqW5k^3m^_xJ}_g5Y2|u8+~)KjmUJr@Q1fydU`MhRR5)uAlj&1`IBfP z-w7P@W-qU%_$t=>o6qBLx>>}hPbI&3S3>ZSFg^nQH{3AE--B(^abU48Pfj^Gg3Sa# zLC=GTw}aPhXAy0me$siAEF{kLZE=iKeGhIV$Aj0U%&0(n69XY2#Vr6h@$7r3`>9rQyKo}MA=2Pzn`dd#g2MW)o%?-Kl zWM17^^9-~vycaJuw2f}wT4$PyQJZ_vP|0JVNJ~wX&e2K@LEL!LU9Q5`QH%dvu!j! zJsGZMttonp*rl@s)A_DsyCyTkXycyVYE_(kTZ_y1O%9?->Nz{TJb+IIy26?^N)2I8 z;RGMMYGPjh^ICt8+sbm_wi`R~(*9gvhBFxS-jJ?Pu z1a%`9IhCFwvuH6^PLh>tH#SiHrTOP{&CPpd=b3~onOi~FpFn4?CZi+HW&6^z zdcx1}sHMdukS{e$&%2T{Ub&v8Dt5WKTDRblQsfn@=NUseigOj|W)~`MdY{DgHZCZa zSe_UtGU-y(ESPjbecO4mRcK&yp#?=kF}&A{29;i(G|Upv<-H*e2~no@^nluN+0;rC z?(~;Gz0&{#&6%L|@%{O1H4sA7-{cn@y}k$bq_RBrfEHe*A^#27MO+7$nO!zvVt6-F ztq9i32N5t2xnEWuhflJGOtDqv_Ov6r#@u9y8wacVHW99+T^NiJ6ENy2^LC3sXt=c{ zeGD_}E0&%SP_T7m^F=b1eK&OUd-7P6a_!;ShD_lQuzW}EHE&;>H-09pH<7#m!T1T zn1>xGmjWn4ApvUtpt)~e=KT{R>xls~gWBiIfpaF)g0ITwr|+wo!yl?y&L{K@IWOw4 zD_^L*>vZAL!v{lEm8~f<_=wKha&Izhp>b!}Ao|#=L^*0e>>ILKdn}}-BQgNd&{l%i z!_IJ1vR0j-x@@&|g zWnvi8r0YCRzGYn^4ZLHedXhk9r#y zfdp^P&1U&c3ko9NI}fGjV?k~sj0uUt^?*kM9paPi`jn&?xCy@ZV~j9VnY&`MHUQ-` zp_}42(EK%1x6X$uTtmvgP4XKE(o!6OaDD)$r#T{F)Ce;o9?+W;WSuEExqa$|9PT6?UP@g&4eOD zUxF2%QCCsrpuM_YJn$+8t&m4F>+k%Q@f*iNDbr&vGKdVaB6o7HF=zl5nW5vK zKGEH@w(J8)jqYsz<`=Z7TYugSalz{kSKr0=Y?K&yAmC!lNb`ougG`d|@-g zewlU~_bAm8YnA8xiQu^H2h?*U_R|Q0e1cP5-hO`2Y&kQiIg~?gxaOArLV&pRs?=PY zRns<^yHSX0ie*KAZ`Cu?{x!qA+cK&iAcBF+v~|0v98>K9ID&_GlOi}6mc_cWU9N@LFiH){;b_rFyK@U23xQWjA6y}%e1YfdCDnq8 zDNXyv1xj;eY}^J(sS)RaIzKduAJ#!NC5~Tu;8ZolXP+`7N-?Z`f`;KSLJ~|hqu(e! z$1AcA?;F;@FT7-PE2MQF8J4|{X5&2wklb(nw*n+Nw%HN?C?HOGAgv@WEG&uqs|iAj z<;HWeR~XK)Ta(c3I}qW`44biN3uF*m{4}FcHUR|puZyKRUN=w*{k%Hc$0--KeX%mm z=_9ZI{L2q4y^d3LU1MvboU-Lw(M{u@qE9}XTJd6X*4}m*e8&4o+8w+eqW-))jJsC` z2-ia^hJD@&Sb(*(M^ngl$a$>WLh+#*r{VeVJ=KQR3rE?~LL24*_YnwK8 zTzz+!eyi5#`G!8a$~c7B{ks}wX+&Fw@$MXMadO#rEgDKI7mv62z3RbF8WxF}_q>VU zYwD@94b^dXmFjAnJD(q@f%9ne5R^Lv3inq zQ)oBGvpJ)%$B-X;fb5v=;!*bB$_O5(0CtNbdcpYoqI%v7EZ!j=J5UKR+O=vxoa|B0 za($bDH?vEaF3X3RgG8X?@3H9Nm8 z4ISQYznk3DtcILibWSpar|N7jz3_1R_WFc!-Tl_GS%o~inXsF+kIq+2YnS4}pI&Yf zkw?6U;#+-#+6xv!M z>H6jld`8CxHl1Ld@$wjO!$f4nyR6(?yH$sU~kWz&$9cyRu&ceVN$hQp(3zmdfAu+x*to?eXn~%Av zjT|7Ju%Z6}#3K#$n3>zOm_7u(GV+`%8WT-}x*9NV$P*XbDEV4sC`HH0V)WP!>(*B%Y!VDUE!_E`IRX%+m6b0j{GU(b~Rt`faBdJo}SLaZ?_wt1+oVi@SD<6`6ET`|z49aN609=RKv! zuxWL?x$_0`SkaLP!tpj#Grp(AN|qNDqYh5MxzNxdm9Xq@+n%>f z1|4@8dDyY^%UoB#FX=CJ08pC@1GUL^oc@)=62MGp#a!Q<;)!EjlynR4^`P)unV)mZ zGZ?T(KVY&^ra|XQl8mDa_3uW?OvfYbo5bB`L(Vs!%HO8?^zlukPoaFup=0!_x~e@T z#&YBSEVM+=!7XR>J+1`!i+YW?gZA*V2JlW*yqoDAm% zuiIVPz?2;5k*(gntKn^X{Vc)9m__~*AH$)Zc2TR?JQV7(l3=g5n<&2`B3h>HdR51y zHEx%Fw1_FYa%b*S(0Xft_L(-i{aX3gZ;{I09lm05$bgTdIMOWN+q3q@&1wN(Q5D6a zxY0V%0KWl60d>=iUX89yXM_5sEQ(88Fjrz1=hl^Q^P^N87r^k3ApZ2O3s5>Ttjxk# zl_F!!GcA>+8%Rsa!wxIj?vvE!MDHI0oOEX}3Cu-Fo;u2cMiV}ftqav-5@p}g-)+zK z7%}H7?+@xKugS18Vdg=pl@pO%DmDG`osfI2wrh&u`pmzG+>?uWoVaL2WAu`WlS9-V zTm6CYj2-LmkpFli=YbHXO2>r}fd6=U9=cgw=^-U{;T0*(3GlKUK23!O^|AvsvCu9; zAzvb==fQVJvaoJccD-qlk5Hm$TDD#R)iplBqJbl?<3-O0pZ{K^_zfZf%91l~+%iX#A1;Qm; zho;1j*c_4L+<0~?bj7`yvt>*WI{ zxEp5|+iaf&O&AgyF2OSxEWd)Np}A=}rG&VHB&$?J1}J#0hCawc%=J(XQ-iV^BG6!O znB8L_#~af@NS(NhA-I|M|08Zj51a9`R!dT|$K*fE|6K#FEq>L9qSdmbjT+ z7uSuwoO1}(x74$?b>&}&6TtG}`ZUzP;s3q6O8CQZ9E1v^yc6NxS^mv?CvKo;(T=}P z)B7+Fa(H&3i+kV#Az?8%hCIZtDUL+PE>KbelZhaQ?)lu`Qta=4*CT^~>8Om)iaqzT zJbM}~%kVd^8CoFXfgY^l3Ap}Nol9#9U=xO((lGtiCeZF~g4Msa3A`qH8VjI!8)zpd z$SVFtyBqkS(&M?=$xjhgBg<}Ss% z>?5p0LVtoiYU_AMBwODCK$WgM_l#~Kb1qVe!#`k1&VI@X&E% zmRD(P69aBwfbIxP$Mnh0+2fndIAAz8gtZ|f2i{9x#%7h zg!MA2#1R}G{ogzZLRt2&9NwNkasS~lH2|X)BXC!rk`{lR+;u2(*wSr+vwiZ35zNM_ zP>A zc693Dz*w5v2NzF1qZu&ILr8j}FirRL&=?{c=fAiV$Ay4Fc$`P~9C11vp1+kZD!O41 zuGhjCfDlHBm*>BO+aDRnE=*F>yZ4>qH)#yF#x@xo07I&<3bEcZXjOv>jKOaIc`E7c z_oDAx_*&QFrmlFrJ3krLyDPG3gQX>k5qU(vqgpplm7QHP?`f#67*a*wbx#Je*EK$V zy#ISJGU)5f%qrE>bGQ{!P6UFHIrI;XGQ864Wlc3By-l0p$UkwbE7;RcC#i z0UgUw22uOQIH=2(V&KAG#iansi|QE>pw)RriXA(Igo9Eso2bK_HGl2{J+xBlllT}f z%lCXRjPGQ)Y}EUJXNbMOTrO5n0@vz+`G)-O*UHypVEgLbV%uQ(Stw-Pd%Q-stpN4x zg&UM*X^|>>-ZMHQ9^doA1G>Nq;Kv8ewoyTkj5FPw1SFMtlrN$SukQ6%Ypy=9SeE&O zHqt_km!Y8;EAqa&u;5Ai5Ub+8bT9SBlt=g46w|5byolHMpk-%L>o$x-uJ6 znmenw58qD{Nq;X7v9i+vK$7k?)|qf26Xz|~SNy~-EbEJC@hwHazgp2gBIP7`&36}1 zaozdMo0F{=MM<)f;AjnJ1EDvQeQ)Ufn)myb{j-3dzmxqOY|~NRAKL`%8bSH+udJ+* z(EI9n(~#?N{Oeh5<_Ndw19;aPbdA-M)Q2Aqv0f7&`b@%mY2)A%P1ybDw;>)nmD3Jc zY#{IuuG%y{W9u3+<2K*KnI%3cpA7x`yZ0dd(cY&0k5$p~{V%NwB;^hLOOYkp@){SN z5WDT}Qr#9{LpI~}MWXwcYz+0SpEhmu>}Y^I#qnTd27DxAY9X3{4Uh1N?a49s0y2Fs z0En=kN4q5se>=`|{9~2E6GYV;a(%re(gXPx!LQ>j9ud==!4tjDV9WilX8jM6BJ55& zu%>?}aOM%(!1j|?rpyPx_1}9eJ;gGg(SijYWWERwz0Tn7q>E(O_m>6TShK``c<_JG zbOAiHsPX-^{k$gw1(>GiEWk~oOF{z2i9qbwy8+m~3zHLcy%HpnNQp}`GNvl0NLIc^ zM2k^&#s6h8LGa*bx+$}ngO}xE_@M}ZVCDqx)qDK`wfwRa4=e$|FgVlG9p2AabMVQY z4OlTdxwJ@b#!UPU&K$9P;rEL?gZ@j@3fy8L$Y)T1Sfb*eYd%V=Tzt>Y1gpSXxLOes z!1UoU>kRdWbv`Lq5IRR~-cbDoy7&!taLeoxaz99r;x? z|3Ak2|Ag^IUe4(<)4nk+kdXzmd=w3)HC&+v`L7PoP3#Q&n|MO0v0oqm!@-dK>&X27 zjDvZlL+tYGXalOpu%nILb$K})bT4|b3Fi-zo(|zp*I;8f51-=2P_X)>+(`}ETAyHQ zx;{yI0$j9b(iuXjZ{hS8&l+Dx9`~kVQ(N*OX1S>_NgZ?N-|+$jy!JadIsk^IjE$f$ zt#a8PEoFDM7K>eX@Wr^g@c5?AOfBA}w&PqLFWFRs`Yh%n?zL9BtTYBC`WU=_f;RYM z(ZtlEFC^qe$LRd+vhc`@^LBH5gRxQPdsLyNQQD?D#K~L9WhsZu=>#&unZmikD8^fg z;#BZ)=Y=n`|AUBBBf(rwvO-6DCGaSfe_L`H0x#kOSRV)sJ+aFfb%knS+;JYguk3Nh z@XBOYz7Bte?D<9L>wjw>=2s*rO20jk3yN#aMs9Ph4h~&AMlba0iofE{;TW41_e81H z6Mqf&RTQ8htyhZ(#B<#(xXtJU zp3e#h?jW|X7`jHQJtQk}?%^&P%#Hb}<4-E3;0~_p-W#3R@dfMP$GgFW=J*jN#3(=qpkKyfL=* z#fi4i+_u)Ex5n<2+xztoEOa)C(Y-%uN!z`HcPQQh@K40!e%(u%FK_G4HKkOQ&sE(L zFdIB$gu>Q;y;hqjiwV}>*>-EqS?Gv__S?8B(sqkFl)2j$Fj)6n zHE4H7^dfLFb~LnuSKZj=T$=b;tGS0FC3T=gjPGm##6-eKo{ zM}) ztJUQ)7Xk-V*(t;GLPUzhn<1&eZ#mvkbErx`C!@WG@1I)`d6U{U7}TZ96o(mm!m9=* zB)N^O87Wt+o#;*20aKnEaTwIOikQ$I_qAcssf9bm((tsN1m_Crx2FHdR1ArF_;Zp) zpk&xo)+o~YQGon!-TbG>UuHvC96sHi{n7(kYODIqe)xfNu}oLoREGRVH1M=&WrDx( z|Cvlh+}G2Xa9URDl5IG)I<10m!5yH1j(0RpB_}2}QH8^~cj>jR%4$934%w#NIl9jS z&Ya^!=rVfSnD=)lL~85Rg+ttZ0e4ISbkbP!A0jy|U~!XV!Ta&LrdEh1rDqza4YHUd@J9591PcXEp1QHMWrR_+z!iLli6NH!u037 zY{tI6_q2~d6i-A7?wnsAk3MMKrog>aN=V`mMJ>ajfwlTNG0oz3GjW`Lbo`;CTy2hD z-rllXp0hyW?}FLyUwQ{=_`Q0C7d`|hZg+GwP1&^sf!1DASf9SZ0#fU5lyd4JJx*S)&RD_qZLSkDvxsxzlbL$4tb@&@T4GA)%2Jvb}I+ZOV z`Ej<>j@cpHS}kJo;029@;!Sc#_0y2;v#q*Gwlt}5qz53uWf91QJ-H9Owzg2#PWNtu zDO>ZifhGEMz=+noAKavA2!JL%;qMg3fyX@V&?31(|CfR0{+isorEqrjY+ZKPozO22 z;IgRJL^+hb$#(amLp`J0(q-H<=L5U8x3@?!fl^GCV5%~pL+09YaJz5iUYncm6jm{9 z5yEpB^(9r(+qHjrTcw?2(#5lGKHSwAepIqiUCv<{S95pj^ZfezAP5{mBZvrbz2s0N z*17%e<2;K$S8BEwH3`&0(5H-NzA8PvaN9(-)LLo^uGU>(^v-V9|CCzOOX`2%^p(GJ zwt1gt1ML5Y{vjwheL*{Xs8W0kN@9Mr{Q|e{=>^mTB|8wh?8sfyC3Ijog9~iXVK43g zCYsy@?M>DhL-(-zAOlJI2@t9;5N|fIsRerwKc7_b>w>uwuD3CE3TS`s4Z`n zlnAlHA}_v^4Dek_U85$cNbu_qlx(Pv5#}?P%<%{eGg5h|B{}mX%wx>~%<_L%=q=c4 zWJZ4#48(HGkyFK(b_|uaxmjbVx`b|=?~@D1U!)Lg=g-R)z&atc$|e29=hny;w>Ia^ zOK0Y^2#npp(YcDsU~LDT+QurI2er@btQ&O9Vil|hODE(2ZN%3&-(h=w!lHC>&c3(N zqPWh*0kGL0U~QD(M;`&s79${WZ4sDgN=dd8Yt3YXsi@hqsplkqiFh-Hn1R4q1U{b1 z0H={{Z4eN2qE<>va_V{@!sQ0djtg|L#1-#XuGA}#W6`^mEO6InEZ^(P{3iDUw+6-8 z^KfQIyD=f*jTBs2w&o(8enSgG;97GitX*wV&2!`ajV_mW2Z@(j_8hSJZ?vP~JMwi9ApTGna&HOP&*3z6>%1#(?z&`z^;2&bEVf_6Br8B$uK z&TLlt&TkP9{6jtPUQH*Y=u@xrR>T$f?hwMjb9>r<;QwD~f_TU$=et&Y1mvrKsw&vk z8wid47c3BjXhBnhhKij^C9Bay9QTP_KvxcvE?EH_`)#f|kT)y*^z4siNLJLj`@mM}Q2DJ63%nW#Utlt~HO|#~FSIFCO_p}Y_rmI7O?8f>|r8EHz zF=Uf<&VPzJyR((AL!hz#f6TpkIMnU`_irhZq)3vj5QQj0)*<^&4B17r6 zS;xN2*!NwVtl9T9vWz{6R=%6zRo!Zj46>! z;ci;-)u~%E6_)x2JGrd`-fmD2Xb!HGYf%}1E8W`ZLC+otXa`685s6eesd??=c)<%& z=YSLJiX0paCYnv$CiU97I|HI*`;~M<;gJ0!o4xy=u~B-c7sUA;_>Rv`hw${S z1n(aB3YzGqG<;8ObL)5Szm9Gs$ zAKZoVh&%qNcXA_Ez33MU0N=mFSCD^Jf_`;x(&3w%M}gd6T)RMVq{>QJhMN!_a)~P6 zHxg3aaWj!fYgW6o03rRXcn3=d4_p4>TF=OA&d!1p z>Oz#fyVgOjC7davycX4rKo6dOuimGTuj=5jTJ=(qytmSLNoW7)_V}cf&Yet03@50T zo+FGE}vwHG4je7>%~ab;a1zN0U%YQE^i<0gY^_2to3AWY2TjcUO{-(nWZ{=HSMJmp9_a_xA4x`(BNtbKjKP80XB|0%(JVl9nTO?Euu5_&AMp8u=KbP)*-X_0FKVSz;y~ML+f&IA3GpYTh~bP@ zr-BmJ-TjJWU9m~C2yuV?P8>&0@k@}s;Mn(#ohw`vv;UpUoZElrje?a(R{BZF~8ui3-(ZLRweCYUM2@%^X@s|CPv8^*?zT94k~ zvnyUNHZx@wowEle`ltalr<)J2;hYlmS%ICY8@usQ$#Y@$UG;GvHnM+;z+WE4&-2Ny zWW^a;iNOLB4j^Amc*cE_`N?D4HS+S4r$LdmGUi_bro#aA?<%}KojW^EK+x%c7-U-+ zE4jq-n&8mqQx76-cm}>?cI8*etoaJDckyPN_`H#))1B(6DyOvgnb+^4Gy%^45Qc)Y*4}q{mNr z^Yx462RMnvvY^}LcEa`&j$PlB#^-L7bCqnOt{3R{&t$bdCMHb{EAwDiO~^+!G@Zvm z5Bze2SAfQ|JH?&Gb+>ZJ1HAR}&Z>ja5rLkJHbzA*yBJQWK2J63wzc2VIJPSq35(z{ z>cwbNw5as^dQnEj38+Qm4R}I=bsfTG6G?&=VS+ZJ(VzH_+&pgwh{eYzO7)24!|$*! zzv6RO!4e}6aFb%5t2;^6eqEAnL$Fl7a^|Rb2Zv;?5>aHQ-{xl`&y5B7)K5y~S_g|u z8LCSL{O5-IufzS;A)e*ik(z-V=F~I~@5@7TA>i(Yj8S57*Q4kmOoCl4=t(4+tbSia z^3Dh>=~55mMR5P@SUMNJs5W71g7RiHF&-wkx04c@2*C;e^~$L^V!~dae0o)mcxm5|1Ve-; z7|(Ni3r*D^L>%ZFpfrmuj-UJ3jdvg6cDD^0K4EL1IlB~Xt`dU|18e0)hEU6fGT)GF zi86Eg4;2pMEP107Y=eCb#;YLU4fDSHz+~s!2Vu=71}hxHN*nKs9(!h=Ysw^)D+LuR zo@T0Jh|LGj(EiRi3SHDg47fT&7kI%qBYZ6~C#v0$KHs|;=r%vup60JX@EpuNT?niE z7{BKCE$IZE$c4L)Xo+`I8z+@1+u9v<$ID4YzoJWA`_VvkqP#xcm^X(G#=840S8?z| zN25Dd%&F^;yjQXj8!g7=kA2IaG__hWqx?_R62+7YXy@;A9RH4Yd1_^N(v`5jqMt#f9+C!CppISc66S>IHv$&@D*z~cgp9z zh-AaKb&#;R~@8*5-e*4AG6ulAL z<%gdlJ~5vM$+|jqYwIZ;UIDcRKSzb)hy{;J>Mw&71UzHnryD`ffj#ialNW}P7qU() zx2D&_aQdp>CcK$S+bfP@w#faHD%ywFLzzWo>7RP6j$IJ~=%PIR^2qr3@}7(8+shXh z9|vwiYwtM4-kf3_Fa3h-%&;nGLYIqo7QB8}u_W#xOsNRS!ypAIy+5G?+Nl@dVP1=7Xnu%K9obr&8L~(hz259Yrq;lgsHKo=4EW&?iGQ?gooiu4*c7iq11BpbmT7FG=3RF0) zR3%@O4_t({Sm$BIm9ASiNJQ)<^NxF_(UJ zH&=--vw=Mf&hEXnM_+Y10pX(xbhZeW_Rn8IJ*@v zLHt0`nH{{Hpwh!D3c6(bR8Kw3=49g16NRe85gr0#;2_)@#q#>cM<0XCy-4*Zq7`8+y}n~Mr#(zbN8Ii4?+8#puUE6eI?=R=2=V#w6BIkL&=N1{oR1( zGlCoz`plvZQSF`w;`pjW^<>9Yev+*z9U5;aopn=~U%FmMK!j&;@m>0%6;5o&_2c>^ zm>ez3rA+)~cdR-Vp1b{AFs5TjT+WdQ)d2W2z zEGvHw;<`!B@vGOPRkYtms*7=1B}Kr#_DYxz<|paqYccS4B70ZX*%0Hb5BvsG#(|#Ky(9DMYMUW7^T5Ob z9C-aeDfPp>NJirB=1{(%V~<>1re+V*jGw+yAv)@og~{?P5xkwP5$A)$5bZd%u#D#- z^al}81Kl*HO}Xl+J8qyxr6yM=sM$0`Q`R45&#gOJMAMJ=Bs*ApWGw)`AM`xyftQ( zy<>Ze%}&l}&Dk=-YKtXyozt=6?v?oo6RCQ(dS+1DgUN%>A?Mmy3Bz8Bst2iaKN=yNlu>2tcXtW>iPl$$#4;zEPDm~Td4S$o|H-BgvXk+b4#Oc zwyG0sN@@jZbb}_{=qTr`PE#jKh?DrtYuxO{R^0L@I zkwsiPL0v&GO%9RFV*l!ZWwGZbdtKtPdT+eG!jewK;>&GI)tDG#CI*`PdeiIH)2>E1 z-=E$qg$8@m@TO13%A8qS9_a_?ANeLD&YKjP>fa*vu3g6~RU;?8)G1FJ7)^MdEPq`0 zVCp@-Xaw#^=?A8b`d0ufQJh6>tgg~eJV>j*lF#`tJH9uwezmeL&Ct(?)zkCisaRot zoRzj#G#BO6ZCO66Pf`sDQ%MNDZeA#xizNfPjczq3js|yIw#SzN*cd}M^pD^jNkWQ1 z#~N#(%x)c1)gGkt%Ow7rCLGk23%r~K=(-@|1bLygz`-R&UkG(uR@A}F1NH#+y$`orrE64yN%w$6po5Hq( z`6`fuq%Kx%UoI}N+m4>d9Rc12R$fYG77*|HIrm2oqJ)=idMG^_gv?$s-}q+XOKHMb zGQ>H+H^#SsP?`dP57l*}rU_)l{JM?l8o)n*8Xax+6*O<*$#X+Zd|7J)G@uPot6m{W z(XPh2g}2pC_$IwF@x$qHCTS=p`DQjmaqB5DZ)$9N&qvlt9 zGi&osbnUoCub)hGhz5Lwg{oZ%h>WhIu1n+jA^5{(I^m^UFA)mN(PxhJMi;nr_R>FN z)ix`8GCDWK^PWIV6hI9-FBsRU^kT6L?bK_f5}_-XS-?1cYCarXUXX;m%(^2q3A2b@ zcPy+w(mf(_i{_G2L2>;jedUfAzBjaG1K~JlzKT(*IwS()sHV{=)f2+xyS0;rt!niZi zrmeAlVu~zqjok=Te)Vxnf0u>}N(H?rUQ@8QLw=|K%0xf(>S|qs|K8rFv-rJrvqp=t zp(|Jpfff3y^*MxQ+H0}F-J`ynRjM`d+~!_bbs0|6a08(S{VzVT3mI_%V?kHosL+?h zego|$)ZpZqOHl?GO6W@6Bj{u4aWvFC$xfVMsKODRtB{q~H0P}@2bAzUAfJ8rK-V+|)JI{>zl%d|DWEdve>9p3v^sik3 z)!VS9M0*Q-ftv?^1h`ue=|nG(-`U=zY)I{#&4?jw@h-8HsrJ?fM>)Al7R(y@G{YMm?a1i5V9Cneh~Ru&!;p z`fBjhpj(^wQP)`koM$0=kM)J3phMm4N$b8s>sQCRp_|d3zL-TQXuH^fh*hM|r*yR|+vV-5SbQxI3Q2jFC zbeUuPhbUK%9cTe|DJbF;o*iG1r5*TLN|`n6dn`vc6iw)p3}v*$fyThRk$sw4d;aUB z+~du^crLrlwH(p`nFkCR;?I412dzWbaLWWnzPN>?ZXtL0lDLLV-%V)J0`95LUFdT~ z^kO&eil$IX^szzXUP=13exc*5wKa(nGB|FV$WNcsWrALLnZ%Ak_E3QBFfF+x_iewq zu(fg$Ilp-u7Zs;}s=ufQflhH~g#&O79n1lWW0>n}DRDdcdcfW>I5h37d`?rWrF~2wSTqrls zN9X7&v9o#ak}HjSf0oKH~1c4&SuBU!m%!e%yNxI@u?MXY2INk~D92QbG0SGv2=wyEx8`_VB=@$gFF zRHT=~^^N`*7IS>h^~U|KAq2u?#HOU-`GYP*K*Jq~|MRO+)WTgwAXINCeRzd>cPEf@ z9eK)aFJaqC-$diw>donInffjTCv=y3ZgW@~F@-7BXCyVbF1R*8?QsfRT@s|-jUn0? zS7?sdgeRMMq(iQ*?d-RQpz)PGn2PO@2QqStH+EwElVxqh&g-%J)W!OZ;wPaDy#JsXJ`^jmq4UdNmK*t^B1>@X zPyr(({JS;;vr~uR4+_^8y7;xOuTvz+a{STSmIkd}xt*eRv;R0#32Xq(f{$Quy{|MA z!x*6$pHz%i)FQhXDCo_AxX>Lf=Qu**-j6_}bM@}L%784=cY`i-tXpqUP5vF=TRw!t zVE7~Y%cOiK_9!z<4kjYrVeiOc@!gmBms~pBk+(D26pkV}L-}gCO1uE^fYwcV2sTVJ zd$3lr(=%U{<+cg>LmHqz^)Zk zEZx;`TW8dKa5IVk&9BciIMdZhquELX44fw{^68!fFYN8ObdYzO3Pf9!*NjRU?+&T*N|)ajQ@6i8vP4s(PfAR5TYf;%k_w6by%% zZ|0?E6MJ;>pC;BTNq3568cqgBn%Vp;D;#JL$Y98vzJ`amZWcA^ zyu2%I(hn8Vo^^FAX)vzaoavTN8fYqaaEV+l3|(DxGmo6B4h|d%6DSd|3PSfO6^drg3W!wR|vt}V;25^*jxXJ$H{hq_H#_cV(Xzh zW;IZ{Iwx&-9sk|2GeBcT)swvr)oFZ#b1#84EVkOFYc=i8jIPzc=1B6niRhdtQe8#7 zY5`2T`~Fqqv}@ynh3ao^$^qI1HCfIN9Mt_Wn{{=GZi5c7uXYQg2YRuP1J;G1DdQ?E z!q#HBa&Y#u;#U^)Y>M4^=|nlVehH9PT+vI7x?$s7mS34ag3pGs;Vw8_I7JU~&^#4U zZ-r@;uFQSK|Me!CUpg??@-a;BlGP_ClvThz=)+lA ztC)%U>0o4rau#Xn#l3fd)y?W{F&58*>y;ejFXOD6R`5v?dqLNa$Mx}OZ4K5gbQ=UHB- za2*o&=rsG=YnHKdF;@+!Xl%EIDV8w>^~NG&Dd@T6*I*#RT9Nh)bp*=!bl5%zu^<^nRt# zMST*fh-;o#dl;48Kdo2i745URqFcXV`R&1}%XgBK6Wnx+uIT4ha_bc<;0wbK*wkMC z(SG`qiV}Qmeg>*}9H+~Ego{OZAPWEw`e!#MsbeBUfX-{3`cpF?Y^7qC6X!mU+<^{^ zHdBV2RSK79E{w5~NMbYfzq0E9hJ+mY?!T= z7P-1`3Bi z{8-XX(23m&6H9$61~giN$#9R=qdmAO4m@PC2cUFDY73sk12!TaY1kypRj3|!^FwPz z@{Rf;*gci9ZU7AvE@}@$_Z zH?G~hBnr*$-MsgnG^U5Zu1oriHNIVC9ea85Glur|)X5C{|7chF?;McYz|OIHs|cVU z`R;w=0{S_L;bMytjAMv4po<7*G)UM~s>EVnGfkC_WGNRU`C_(ww>@6cKzIj7U(k=U zRg~6f#WxAr-QmC&ALQID*w@Bp-Sz4$qrCX=(js`U%Jr0hw2m9Qy>#N${ zLLaAW;&H?*t1+X%=<&Y5^y)A#%0ipw!+a9%L0&%M*15XL{tKdv>3(4hXmTCi(7CdRuGE z+_#uxTHiQV=Ld3YR%jDcINN;FQ6y=gOY&*M>{tojZ<4%!Tganik7sOFx5U^Di zQ4Xq?fU#?u16aHEeY%XcgZU8+2NOK~$id)qL*Nv(aTwB)-s%Na3ZvB}m zyKvwA9JD9Ty6{kw2!24*PuJU{$>EkAq1~N|tUjYCT8YmIbZPL&fQC_Uohq^>ffcBk z!Cd2YPm`a=9wrT-byY&LrDh)2{9cbMT@8mAu9m!M&Tl6_TsKU>F0ai{Mu{4626q} zmg%#e*(J3o|+qCID0!hQ>4Qb;0r8GAI~ zJaDyUOY>6?n%qwhj?c!YG2vXA@8u#(mEW*2k^vX|ts5z*YkGLfqH^;u$|67kv9$e; z`TSkM@bWl_thJ;fuKQeiD}B>s9V>AYyAw9_OMDOpt7qMEBLYrd$G$IM+!m9Gyw4G7-eYe|d-}0N!F0o1W-hF7pu(lJR_Hp~I1|ZTVjr z3E-rTWRVTdG|RRNFY?~z!CQYUJ}>qwHnrlFjs7hx006`0x4$D_dMX# z{s*T(@FOyIcMIdmDwiyPW(|8R4s8v*#{jHUGTBG0Q%#7jU9^^KczG=Pf+|{>fSv6# zuu^H@r+t4&OW##dvw|C_D@%*|+0eS7O^+`tTf`W5hMMCEB0n>NrlV_qE`chj1N0EZZ7lyH?ml zb@$&QHjWblv^xOTME5^HiTxc^C8iEPr!Z!GZf#*po&Vy>9nP!9HY;)x_Y=zwOQSQe z$b9ig;n=wN7O+itm8*dLqu|}#7Ya^&b2ecl_BPg|2<(Xb8_>psVXz_C=@RV-KT{|hvoc<7)Q zlQ0NF!lG_0!cw>s%5Js(%N13X=#ak%PD{h1g2|Cm4UZ&uZ-8e5gGCFr_CE-<|C6&R zzNJsu($LTNax6V)c-eft#2<-vfz?;~mJfjo5Fl%s&x6BW`Kj+e6>#urDiiW$Yw(2C#OR=i9~_g$ zUmTOerB-$Qe_*KrOxl+_ffno%kPoTadt$0R0Db54f~qVo-6)m8S{=BtsY=MC!+$I= zz>xR>jrpHjU;yZKbKq`lP-o{M*9PASWim;V$pcb@%km@C)RGM!?cR3*SX1j;*s5w7 zJ^>+`S#~UCEicYqv_`m}2RIS6p{rB^i zYy=c*WQ2Q?-Vk6fSyBfu9iAKY066pqBhY(!?A(xfP~pF$7I;2;bOGNwd64^oj~WiYtH}vVV#nW?P%jsxuiC?-8CJ@{{sSGcW zBH6mbvMl|VkoQmVk)-j(FSGR`BwMw`=T9c$`5u2oYkesOVJyQ^Vf)8lpqGJchaaPzP4(4Ebgo(Fnj06DOY%I;@dp_*0j zHX}mCyiq;p^`mJH&?5ihzZbcf{|~(W-8;lJaXYWy4K@REkHvV;3q7T$OT5ZMp~A;- zp%E;~K|iuy#ebLe9!|#pnWq7Ds-Njl@qgXSvL55!AS^Oq^IrIsQaSHth)GM=+$Oge zc*&aq){nF4+Chm2lc^51Ewi8AP>Abm7PFL~W2tu}3X?AcUw;0mu!r_+#&hk3I)%07 zCzrVt41zC^KRL#^Socu&9Z!!``t#)5&yHSci=k_;5+0sm8;+c5U*4Jb{Bloe&UbUy z3MAdNvFTitJ$p@s4mzLDDAwe*4z+DQff@H;>g^*i(gZbS`z%XVE8;c}W#4wl)JzBx z4Pj(nk`SIE_~)PJXNlbB`cJ=Qx~lsDG9MKGz?u9cE*U;{WM+#AM~Q7weG6GT{dSev zR*(DcUX<(=bBJt@x@dhi;j*Qpha`ch&NspO)Y%rAoV6BdRR(q00ZPKmB;bq&|Q#6F4)GJ=2$4m#QrOyAmk&?Wi ze{3jLSDwJT)R5vJfy3c_%hj7`{1Cky^+q3#*+K0f$s2^u>9_f>o_DB+!kw3RUcpAz zkK$iVw8Ds!`c!%l+cnJ(Ttx2dBbf5KW2T*nqC(WJtIUHG{yb20775ZnY{iH!Lfh*k<Ez*rLpX*T0^oW*x zxbo@AOl0OIt)eBzF3aAWUXE<#!7W!TdLh{-f(jruQx<~P4E(TzcP+q~M2u#TqlRzztsa2|xuA9tj#p^7HJ-ypwpe|?Pw9KjM4Z>o*zk7+ zDxbyVBX4bWo#A5EB+)f_Vu?4M5^kUE%xH_@(JHabLw%YS>W-L%L5b$5iS0wmdsBR^ z5Zfx!UaX!Xc}BbW_gCLB|J!}%{Lg(yAm&3Eb))Lds-uB_i~%WbhW!hC8$S*J^TIL5qX7AH&U?ko4rwtsbwFQZglX z7>y!^OPw3NCMMB3d^t}jy197F)Ptgefnth1fk7nJlML71+#ErP zo9Sw4S4LUM$%Xl#=HY7pXR|Ah!2K+NgxiM7hyG4c53g7o2W`*~sl|(tfc% zJF~ye!Q2x%n8``Frc0z+^cJVd%AxUWgU0iV7}rpPA2ZH#kFwrt?|Fnj3RxgzyJ)hB zYme>i<1BKz&>)3-XcIXUw)>8+6Rw`7E-^0Y*sgTEuc#zaRGmpl)T2WU+IRXT0h(Il z3@MKkgtYKT?s43aGmVdxyE+NE36ZxTgtAUwsL(D1%5M=}>XMpv)kjwBo=}yZM^kH` zi+7NM#3vugBf%Y^VSn#1e7IRZmE9N>b9ld_I%eeE)oGLjt02Mcv9IQfqvm-kZnTP!mG4go+?SS=*j94uWGTc6QCS=;y81s(j;p_)91ced2lGZBiMXSb zY$r_;FQTd!rTgzo9^6HLpq%XalH!Zu`G)RkM53yfoV!b2B(-Wt_^pm=XKUQG>B&u^ z8uCqwX}IpfGw#}Z^@iiVoqj}h)V%n*4CCTN8EvAN+Y@~b!IrKtL1_w8D^I&<*R!tm zzDXYBUB%f9L!L zL@4P`oSxx#k9BjsWL)-jY3t27)yG&UCqz;@Ff_AM$UYsY&A;u*Gv%uRZ{Tdx@<>^t zT7sC=wyx*%JKy%Ba?BTc{rnhiPS}u|`_o)3YD<9+Vefd7;&P7#8J)y^9B674MOn!z zEIXq($|ZCrq##_k%wnV=MHsSK#!_-w`8_@qN4;E%*i%>v>bVIC?&IKPDyEUcm&6to zl&pHOA`Vv>g$4ht!1L6rnFmP@}{m0_i@O88BkQ zm3}41@55XXr6pTu1<)YiNP2E)9f2_?T+Udni&%8^eWVjMuF$ah{ewWI5cIMV1*vwH zTd$CJPrYrJ{tdQb*xCHFA?d@jcMfb*YXfCM{a&f%R0I+sw{>|a`LuF}C~k-fKMTXb zqQFH@$2hg9*YfvXlD-nM)-m~f+(#`|%-?~_ASvMFn*5~iHbp0|Uh0__bmFlEFFSO& ztPJuEs}?q9m2WupoEt$mT8VWoLj)cr9<8ack4Zj~Y;r=|C>l#x^ktLK->awG-oO*K zJ}uPKPUrId_=4RmS}RMfhnP}3;~la5H*(YxRNkK%TKIReQKK6eKv+C+RdC6%>65>==v~glY?=)2n12!lxJldnH zGx+7uY?r#7FUxG=kBCtz@z`*q>FJB#cw(Qycpisp@OSav`eMXKjx!{pOR-Fb`+X@R z%tSzs6e8JDH>q*z4&#S0>;(|`R8uL6+B#}Vs}o=sf|{ex>Ol&KhnVerID8u%D2aw< ztM5(aQ5?RQ&y_T9_glX?xlYzaFOs4;{o4yoADm5wZ7tK;OcZNs&?(O}Cnf4| zN`1AQ{Sf#fR* zCu*E^;8Ks|_r3I8jMtio)zdLJGs+rjn8D`thKe&5$VWg2vKvR{@$NbziDbq_*l>nQhhT z1gCfB#JG0lgZ^K%s)!$7byk4!F!?xdF(eUooor({%9MoD)p zpQKPdULv+4Wkluva6p`3Rk@OnmsfS%XNcYfiBX*b#S9^<&h&I$E6s(TirZ@vM)wC- z6Khp)yw>`B(L7N0%a6pt0aj|aa9(6%v;b|^BvXbc4)%d{a9DLLJ!CS_ge?s)KfJIX z6xl?KSzb2%|NDGDQfOeJ{9D>sZa}yk-k+@~?tLQps)?@@Yc%v@+33$5pbOlD@=&s( zTxHy((v%J#JD$JcqofzR7Ys&lzC%w%HrSB)vh8pxg@A3GJ_Yvv z2QQ^Ki7VME@z=;r{pwXoZTbqy3ab4MBvvjy_C5Qye(Ecw0>$mM-U9lSu_|$m4)?xd zP4j?_O-oNVo53_SUbn36f&5gx7XCXrrXe7n%QFyH57+TCB>fv=bAql!T6Ni5v~Bi}4s)(9>Lg1ND(MwI zDJYykRb_d)AHbv+rV#zaTq#JmVJm;^dSCvEvXbPE;9_qW zhRc}IRUXHkNLGkCiesZvp64v>C5+%E!?97T%=4gseQxLQSZPPHJ{sq^E_2=g-K{g1 z_~uE`Z45%1VNYPGmm`EB$tH7^LC$B;o9bkT6@m$oif-}N2D-chTJDtB!#hu5?w5lM zNU=f47mS@U`0B+FnQ6mphgqq_f^sp!HqbxRO{Pap4i>RRhn3ZT{B8M1VpT@-TT%=T zW~&t*u;Wf2KbuOEqij8~H$0}jv*FS8y{KRVM>|rM{I%&|W>;NIGcez<{@fiSL!);% zpm$Ot@FBirilZtJ0;wgZNj-}_){ON9qtKK2BAS6=rKK!2c=AbJ_8wrG@Pvge4=X9mF0jZ2I`I3(-gUeQ}PDvvT>-hh>&A@J z?4%+PSf40L{#LbVskzuXJGM6-y_x|sg$-O%ok(|5UCoY!!A?TnBiL*!4buhVN`=!* zr#f}q9Amp)@^J0g;dhvjvpiX~O4o$RA5K;{7EnD%${(tZcAjmO1yJc^!A_wJ7wfr# z*8=2VdLWUNSy@>wlVJ?PGTURm%#DpSI!|VMvw;CPDhc*E2HfFzd$@QOwbsDf1!P8- z8;Z?*TBtJg_!Iokzs=9rXs}vSTO3A`^XeWb+l{|Y3UTbf?eFh8eizoxeHcTv%%lwW z-MXUs=7xTr%}!9!JF18-nq+v_#vKt-9QSE5BSI|M)$k0LuFk*vgSx_evXO$=d)ix& zsY}NfE}+SSpWKmrsGB72a~V699A2 z2sNbd>V#~KBy0X36(`&W+*h&|Z}+la%~6P0s>*Sh@92@cbj*8hsh22I@0!O3J^4dy zV_48*Xp`Qz9Uykec~U&>WoMU5#g?Q-N{PGGo28yI?(I$zW+f`@V%+5a#Zl`?QwbMW zL5XS}#MG2m#CwZ3$!g2w=cZ>pf^I3j`=RR6QvY3tA;>O%l!GOPSa z(v3H8>Lw2SalawytMlOdHz|BFs-5yaF`cB4qtq_#fnxHrD~{d8vmwMyNYOMol?wv* z6K9R;yr6n+`~2sH_D2w#HsVP2Pzs%=A9(^{W=rO?jOY>;Hg{n(wE;8NS~C3a@)V9J zk1n1wjO%U5@bBlJZJfeBOJ#-8p{Y&M6dvBUGG-)Pj?iF#B6$6d%&B_v%oI5JDb}GQ z=#UFOv?|gN+6!`pWT<&&pTAu<{|n`?nh{vN&yMG}?ILse{@mowp&`F3T%lIFIyWH9 zQ3dQntt)|)jK^}T^XKS&&+@|3s6Hn7lZMep2S(}^nleOl>&1b4uRC=0glt3d`1}SF zJ$m;|wz^~at?ri=zxTOR-m~{6r4;ytmEvBjWJJ5zL=U}DOBAslI6?W;>KQGvt^C*G21lS8%-i~KhD2pT+p7l`Nkpb%&Hl#(|)u~mRByQw-{gD%tlo61{TO=l zL>@1NG{oeZven1Kb(#?=AQpK2&eJB|9(m~o1+G&`z9?Fz1_fpc97~?$!>tsPhkFSO z;}%k4_%y~hYfnq)x8w)Z=sWT+MF2>;unj-H7`^n*#b_rE@QpmOpi4T}1n=-uRgs{n z^%!j@li>oa5w?<$i2kB23asAW{ef(SzLfm4Jo+|@qxx0~&L5Vt^W6A2PDb}K-~Nx9 z$i32_H&BpnORojrMa>`o^jv@l#}Y6>^8r|TLQmTs!R`by5iai+y>6t!!nu{`cCWmG z9WxagXy(>sTWR-^sTT&~$qu;ys>ALFh6ZHP0~2Whx<{5(N|)?e9~r5COmvO^xTU=k zasCGEy~0SHRhHo{#q4Dcy6i=Q~6jgjl zHp`BtRyx_#@%*qJJ$vT2u<8HNqYJ*(F0-Mzj~L+JeG0R@H`5#lb4J~p%TT~BG{wwE zCS_hlx_77L6J>a<{mD$PM)FwbXV>Vj>-|# zBt0Lq4yE{2n8z2TFH~%!{C@jI!8Xc!!=_2>U&U*V3Yf8{qm{9E8)@NuM`C3d1v6oY$nDTBD%Xj2bbt`_0H6z!41!1{8T9-h*y%K|Lm!p-dFNRDE#JSytL zWCl7%y!*a^MnjN$o&Ax*rJ&}-PeTNM`vrb|qYba{dX573b{EtiU(|VNK6)E;2TNRk zZ<>7iSJR}5s+rn8OFM_}hOa)KJ#K;Y{LSnb(?N7<@ zwPquMV>rV`y5j4;kFFp?3ASD~7yDeq!pEpU8xQ6qMc|FHRc2ecE@+u(D_ThCZd33>{+~ASewv=cRi>8lrnmF84Hw_BuNMTmUTJwB7jy=&*Gb z6E3Uw`2<&cEJ?@k7^P-7@}_g?_ACsOtZQrN1^KqUR=wej*0_6gt*f5D-#hbSU4&TT zK)&xF6mtDeG3eT zn_o18gI=Uy6ioueNm>xe3L`D8*IvQEF5g2rS>B8-BG_8mm_xI@-OD^KOGdanWdgY` zrE*xIO2Th=GOMM{<~0BXzs>}1VjSk;>gwbw-Y+G`?M*F5)1c;Q^fui|KToLAM-yyy z>TtyB7V8n={#v=tC0Y5~VmDVX0%W_LuLzf~Umvz^+}H*g#ht}&B=yR8O?+DthX<2> z)GMXo600m2v2o_gs2^V)gY#v#Vy-5WYS)OrwnM@IISSI)*x3EfpY+Mb3#ts|NhJ=o zJ;f|_IC~kc;h_Fy%3|)|eD@2X2^H1&!K~8K(&UX>PS37v0YdNcPd=QPg-w+$U)C7$NpV6#DD5P?1q?Pe|AHn1)aa`0Z>6V zyy+mBGnzU5G%IPhuF1g~jp_C4*YT|^5tq3eWlDAo_p|=4E8@I`79a^BxP8SV86H)9 zG$8;+c;@!1B;<+(`ArHO(Pgp8ZqgTg{>MzW8EU#n{y+A%>p$0Azc#y%UPQO_-UXCw7`GDPP(|K|guYsZexj3B_QkiZx-YZgv-sG9 zwB9t=uaKfk{(Zso7jJ%N`!J0bDD1C#F&-tqr#uF*aCccI%-Zg69 zx8%r#3%R^yewM^e!F=aA%V>^^sZb_SRmpPY?o8xR6&Grdv~Xvc_eGBAhLwu({#-a! zwpu#WdU!*o^p^QeNa!#Vug$r9{iDRU_3vhf_bB~d@~6kMQ>X94YyBK@R9?Ww*V(XD zi$8kkD z8Ge?=aq~>W3YSr=-o2v;x%}Wd$@<_2X05)i#n;W3*wDN)mlbZJBXEkGWcckED?}M@ zLE<&SAtMH{(3R>6o>~DX_WUDb`)-#80tCun?^1#&)IYAmj#?u_5o~*f=$lDR3baJ8 z+s3sLS(W1+eSE&Z_Q-VpXeC#YiY+WTgb9+|mEfU*yQ{aQRHC6@!sE6JUyrot_?Op~zl) z7s`6ARwzh5rY_23XI`?}b#=;o!CMabHrqLyM*@Ed7=2RA6$ZP3^dg;3m^DZ92Cxf_ z&Az`|0OAA@nLPvfkn8xEg5KZsL*WU<`9oQ^>$wHmxAID#F3Fn9Y(q}9AGD#>AU%k2 z?q=7e_A-a3ig8lz=K7EJwn1S>EUvIVI%!z3Z5H+iNcg@3Pmcrp>D zHR)|-SU2LmB3}%yVFQiHpNR{a_CFIBYah!WR6Ei(9D2r|?#p^aW#|H?ag0b2>~s2f zHMX$4VN9FIG!h)Cbb=rDeHhnS69LP`1 zyq616dU=vA-EO$J2jm)BFXT?n;mGMM%koANvhwMwn}P}TZ!EO97B=G*VSSOGT6n#a zAlnH%Tsx@^3Rr1~8)SE+EL>O1aKUoW?d*biv{9N@SQ@ODC#cjyZ(j@3+AyTd5;URr z#>TZuKd8Isa662$x67O5v`|LR*BlA!vsCq~!iw;Yg9-k@&(m{pSFBDDeZHEs$`oEx zUWU7=oy+-f`2Mr>)v51V-uo|{8I_1nh}Ywh-*A&2;AVxMn*JJ_kk{PZ5gf6!Dz!d3 zxw|8tyPa?%p229TwBS$_S+V*OW(bSfDYh3rKc1Cp6CPXs4z{X@dEVQV(BARAJgf*{ zAC9MfK2uMqP6NTd_VE*BTdp*M;N;-ma{A$&wrByqKy2urzCaxkma>lV zv5H=VqP|tT7r57WWx&c$;M$-MTvortA22GnL1wg5OeQMdJsBQ@ny$4Qb=b&7t#3EA3{w`=;gQVv^^cu+_nk_TJ&iWxek8N$(A>)FZIGK;vhn zxj^c!Go3W_zKPKV7*yw-lJ3kw&ExWB;0QF7llqs%cK zek#~}O|^Zj(~AQ;R{FAh4Cv&>EEg<;x)I7FD_hH&50!$emCCgW)V1<%?XDTqOZ#!BPP;t#iBK$CCZyE8(f^f z5OmLW#p9Z%fEod?&chNE3oYu^<=mKi`jlO)_K=kG7FUHQ54_aF@P?cY+PtM&I?(N68OBTwLXXeW3!oJ$e{v97i~ zIa1kl4DjAJ~|qX)xwymlaNz{%(Zmbe|?4Lxp76;7V!ehXnU6 zPNs&gF9kqEVKb6;jpoKv(R$jh1YrfHD4*5X0_Bn#0n84QJf(KN5;8+MgMw+mJPTDU z!<1uGaCRE}Kb2;!dn^K;KIgh&cizwen)5eT@Xh^CT!Am%_KRqHY9lJEM{WA0c6&b> z))%_6&QcLbEOM?c^yik&2O+G+cF)?-=)>tb?H;9o#&r5fr@QKe?_$m%Y>sWAGWV1; z#<{JH5-a_~bUH{KrnT&DaPv8=`1YnA72EP?|8X-qC(R`6vbelA5Bi@g0igdm zozPM9&yd+QH1Cy3MOf=7OVSImlk*K9IwN_CG>1TG29U`J9y;jy;jcoThi~-$L3lF!ywKiqIu0UJ{@IU&6yg8d`DFw|+AG|SJ^9H;fge8=d55lcfNrfVPf$k~KZ=mpAE)WP~r@5;5(U2Y*@?{n@z~iUXtk{Z$ z2I?drwz8K8ePEY!O_!O_B^J~n{J)@VZGSNUXgf4;{;HVyE+78;z9XN!s8^7su-S^H>0)|-lG&_3hJj9y2sz8)N5Zk!zH!sB1oD* zKNz>Xq10p1zd(ZnPTyQ#`8cVRvn722v)NR&=;x#!-7dRWYUZ=z`Rd;*O^6wKqc##Ry! zn-^lx!H-&BKo##TzIQzT|~XLL$I~deiT(FuQL8U%un+b z;Af6$7sWgO>|#2WTzccYTJ<$D75<0+U`8L3FaNGT_oo=Um46qelp_CQnd`}C=BHQ$ z{0~>__WeYJy~RR=-0Wj673TV$bDjWdE(sUiwauaRg0=V@31X7K^TQ>`JVDws)jEC4 zp!q=eYa4Zc?UCxwYhWwc0hixMgS$%OI^D)_*tj*g0^4Y5c%NqAd!D1={;Jr#o%Y+> z{%XryQ)oV~$P1N!GR26GbF}%9JI!80R?B~@PUv)mx_Z8?42`fA89QvbBLUg@K$|%k zJMc7H&D;u8YL-}LyQ8!)o*7g=`#_?p!csSVjwWw6gp-H11c^c)3FDMpHw{LS^TA;<_x_H?KHb09d@8n2Nc9?jt~! zmwI<|^Ra|0Bqw}b(Ag)!<}n05B*X;fg_b#XBjQx~296C=uAe>xcO@8Rj3|praT#H>APm(_uQ0<9|a@=NDwc_Go!i-|+*CHj$3=rT&; zTcMk?T@<`H;}lY*0#2*tYzVH+#5-hUF7~B9tbB^pHu9Fc zB76FDU@LBKExze8^Oa;18J3SNP=q%k;=JI-9^<;}4%4gC;MVFjw+4&mm=`W0#V#2b zWA`aE>+WgIMh&|uY+p6pv#nX^s zsWu>mNoR|!a4c%D7MNfy4b_G%!pY?cjvX0_}v0d=&k!5Gr=ss+T8fZ zzqi^Sd+%@oT&uj@gB&QZ%FZt(_ersmn`bMjM}BX!@UQ$y@ZbAS#+2e%8BVv0Jtx#mZb!L$G6SW6r32+0Br-=#BVmU7U22UGOzztQ%r=8$KeIcZ6fEg8 zM0iQFd9tOh&F`vSg>@cLCw%tWZwCknIaT$XtU8bt<}m(b!8y0IY<{DSJbP1z7G=Sf zd8Yc($DcT9M%)D#_)^*AZ$lTQ`4-40c5@=Ji~1&WBimD^36PLlhIB_%2uIWhs{Vq( zSA(Wo{UO8+CGn}i%%!3=?wD&~xr-h(Bq1#}_CGFS;Bs2)^FGv4Z?Q89FOs&CYEMcM z_>EMUyNW6Zsv{P!oj)qAv*`EA#3q|GRa=d{%XXgn`p9EFT>zicF1-=R#&1wk8YBaI zQ~m+A>fCkY%AGR0m1p^72{bzi8%w{4V(dPI$!a6=3I0|4bEE2>sTt^|n4|RDrU9fI zTf#~Jn-KjJY)xymwbDkp=k_@;;-2bYSM6@2+X$X(<`}QurKdc!RgZAx?Hwp@t`kZy zOZq2Idt;K|XFmnIm%($6rMRZ`Ea{*8_PgU-PfC##mIMUTW#*+Vey9%Z?d{ziJ-0+u zC_E4CP+&tn;Yrj)PYo?YOAO)n1Sg*HQZSFiz;?fn;wAl(6(jc-wUCRBSBY_N4=sg4 zB6*cZ`yGKQ|DeJX7~%H(7V>0VmxS8~*z2)QYj-1n@i30@^N{sx);t0>Z{+6-$oJbE zY;gg^_@w(v$KK={+yLr0zf{u3#G{-jB)6Jx&{R(~x9l%pQqNHqNtz8iSl7oOnw1GR z>ttJ~xjbr%RobYW7xT@d_B!n4vC4CiCfaEG`5&6j>pxI^dhV%_@k|b%|3b;F_X6h7 zTk4yp;N&J^uGj1^75c|Yt;Hm_f2%PDeOr?kVa%0#;<~(l(QMig@u`OZIaEN|6 zW2V@36^g5+Fi~pEEw3&Y;-Pc1$_VI2I2Z!+80eIQc*q0BXvD z!3CX=VG3mUm1Yi!*cj`%!b1^+0ur*9}v)Zn_nP0uUX_aja`hNO3?u90Pm7_Cgi*oYmP`IG(tvB!!)JU;Wu zi}P!kJv{_l{c4E4bjg>1_C>|x!wmozE4iHBsm@e~9)_PozKw!)*n&fPq|7U@;B+T5 z>s=s7hV_CXJ7>8nRH?qF?;2A~E(oQf9UZ8tzGQ}1jY;Bxc4wLes)=0JV`*S^K$*hs z?1eTi+}pJ1-yx^FDY-eJCyx`UjFdYvya1Bx17~O@x%2MYCvnJ0Z3w#d;+JSL4y;a+ z&?4oM+nspQ9*3Z)fWBCNu5*k%M&1Vc5~%x--~>IJm1W_*_!+YYjc&?2d~L6}hJi8& zUJ;MKbhp?tQ@1Vtb}Y+6^TgDrnYdC%PmOjoe93ne#8|3!y&oigJ=BN-oe~vP>J`j3 ziKw+?^~DI6sAvKR-DP;EKM);QY?PzH5GTeJTYiWpq&VN#u37Pzs!Q{IogfL;^WI91 z#M~^q5(>n<)Ln7wzle6C$uu_w@*ZhwO`7hbsc0rlYc5Oa-`RG8mW!FF6=G`I9dGeD z4!`M2#B!J|7AUzJ=rYU-{4(m^ET}e~7^Ti6vaxY|AI5mwccpG$4A=sW-4|Jk{C6bn zuK>7EBnvZWM5)@MgdpuVC!@7{vNEP?Gw@^hlzGov)d~l~g`s4Qd@pbQ+S?q@h-<+u zKwYbYg_b(9MT@X%rRpKtE;WVVf<^&9hJ^V#ft8ly%KAg1+|G8sk43HSVvBh@m1yFZG1ZMpHC)Uo`u z44Rf}0^6Eu%r&M?`egZ&Q58ejRX4Y9;`pw8g0O6-5r?W5@sAyP(e+$&vk)gifCmd& zV^FmJfGzV!0d&HjYL(*)obDst%7)magujt68rW&qN)-L_s9e(5a!gBvz?=PAheJrp z`E7>YF+v6J z&rb<{cv5nx4V)H?cZ}(mCu)OaLm59?OG4bZ+1VyzZT5;7`Liv<@ST5rdBS|6)0v0d zSa88xsUW6fHxt7^0aw~@6mFkLJ%86EifMNK$kTAzH#(RG`ZZUo|7+9a2--%JX>kmK?(5fu%oUNBUeO)vz1W}zJo)i2YtwGHhbzXvl_FH z>!Ctrvy=>X^XTU-XDWwRgCz(JRVSRE#1zNPo*5}JWrB>g>YTuKyb9=|q-p44o&cQD zefP$J>(h$*N32^Rx)_?FMsQ)OpL`*fI>*kx0PTz~Q-f(6?Hhj9TC`v~d0^^z^k@|4 zbL3bnamMZG+pl9sgVCm3MHLrAC z+R1t&--lZ7uFQvN73<+hM~Gkr;QRoN!3f#bJj?(CHyAhFB`k8~r%A2m-t>4rc+^3GpJSX5syBtnZ00ehNy4e$C|` zVf87EjTunVY0vz(C0$`T$9D>d5iiref4tBqhm6*J(>m0nzT3GLo=S?0<}!*KFeg@^ zO;?+!*~?BYCfGHfjw%{i&SryW+-%D!N5OXZDVQeUHW1AiA-9FI_iY!8y|%W7W$ot| zE&93AJx_tZ6lOb{FYwKjuVUW z(mXY)wurG;E{oTMG}bmJY(oG`7d2%jVj8WhvN@`twT@V6lez>ss)+&IR)~m;N9~<0 z#8(TgE~j1=`hfmiu^--xLm_K|7K)iJi-X-1O|Owzs#JS^x|sM?4_Bq7BBsgjp@Sa# zx?IaI*o57-1)sdw5R9lLmaQ)m{Cw*Cm;#ye$24y8w1goLF=!X+8vK0s{bNDi88c4j z#>7|_T~Tn7r#iWA2sc_AJ%`DD$=?L;gG#?5efCE@Y3(&ximqPLLMGiOy*uw=Ew_v5 z^4(>eez!bS>%CSz$J?e;Lk}##pAY2yc7> zsVB_i`>HErm41vS5)lQRo>5}h9seXxdjiwUX#a|R3vc8YJ@P%8BQp-FK%8H#c-#VK z0H37|9wvaLRv84ulkud&*o=MWImO4Vp!S&a?5ZhODSowkN#y#qUcQ~Xo=ua$20q!;-mVz3-8`7@!3of<*3x9^va&r($uAZ{Y!Sjrsp=x zetsp;3Yxp@{#$~=@MmtgXVs-fCO$&acS~TmlhMl+C&RBw#(pHjK4`fAmI8bAY}*&b zIZJ}(R6C+VD3MmLMbFVyniZCcWOxlmLfE~`H1QwKFzaRAL`uH!R zotofrl}GVsOMvV6y5Ng8b>-<9By&W(GHi6R%i{ad=gMYYB+$B4(?vwA2kUBS&+a3d z!>oI!ipCh&K#yBMmw8yJaPMG%2gx$jxtQ*l*`3m~Nx=4%`GL;iHY4*G^FqkQ*rJwT zGuN8acizdkz28_Gh2GN4)$%Lw7li#d0m$0bNYeW&NrzbfHSLiSA97lqN9A|HjS{8L z3n_kDr+Nmiz9Q@SKKP}>5bwq%#*JV!5?&F3Vpu3r!kj~^XW!lD1GdHbD?{<`hjPY2 zfAl>!v{H~xb@=d*auI?s@1ZAk(a&JUp-WDGN z@f91nyFHX%$_ib>b>T41S595=LA&}U0(Lt0ZUoJ4_P-b=yhZ`H@7OmVbXbmZe{1w0 zs&r7(7n$=fe;{zAfH%h!6)tP`{srr$8NBAXR(zrJtGBsc7 zx?DYy(s1}kR7KlmR*e1OSVgu<^5?!ZUD3K?;r;w(SP6i+g+_{OnfUFxrFCzJvRt@R zXrx;>DT-O0W!h3SyPpfhoj#&#jnBI(I2^ZPK~hCu#rL8l7XetJvhcm>$fRf6Oh&@@ z8bdD#TD;!SN|HaopA~Hzts-xx!Pzgr%(m$kVtxfwjT=p0GMDhviQ{p}+ql@l^?0&%Dx~aNV&yfB*_wx1nf( z{+&@0;=bkQ`d=FffDz2Fhq7dRf7Sa93FY}#zcp4RW+^5?n90cH>g+ZG`lmBBp#@r- z3#DWkE@i~Pos~q&oF$KZ%i8N72I;0%T{R?}>sjC@q}sD`Z zs!CLX-<8YP6C`5~%6Q>HXwi;CidChaDI_G` zu;pzWjm`kZjh64(ug3M~?R#4ktalt2|1w|SNp|tP_(oY2DHuVw(LV&;T9K@dVj7c; zSBHWFKvv8wT_jUP9pw(M$6hQ_RKND@+cP|3DBh!{jj|d?e-FG**`jkEAEOC=X93WO z+d*5dze3z_yJC8b*D>s!p>UVQSR+PTS5GZJ!95?ywh|)e6db#Q z6gLgbAKgT~&K*o+&rM?f;dq6NdcpPmF>>deks@T>x6{`!azPlDQinA<{lm#E9U7%+ z&PF0Sbzb8;`~?-ZdnfqBNIP$Dfq9TXU!FC=FJFYViqrz3JBz)JvK*3)ILexeUbu{J z3{`=NpJ{$KQ)+a#XUyxeRgc0kt-1~D#&E0!x%a3}&OQY!$9Imjk|^Xb)BXO+K7it|Z+3>v zzdtwEC9r+wEL50c6b;U-TlE6or5hFHZ#M+p{-nKqyMdh-)emY|^NT!sOr><~(kvS~ zLS3-mb}sYnn@uWAYj*qsKU!UGcQX*6jd$8ji2qal3^)w%3*Kx@uO_XV8cj#+C2Jub zeJB~!L8v;IOm)T^xtlM<@R>{*k6NZpe7mJL@7$UaB_F?+We`9lIxx4P8i=1*F1->Khy{S2eUwt)BZDR-Rze^r;Zg1NI9XvYCj9s(%R0y+haNWVi=0!e`j+6mV>dhzb zLR$X3+jV_dp(=ugXfhB$PPOBA{`T3fJSHcz4u}yF68#!7d3C{Ta*+4F;v&gALN_rwaB4NLjYc4YQslHoG`AejAG+ySAOklgUeQ(w#WNg48*9^nW?hF z_wBRtl3;snQ5SiYMfdnz=EoxHuq(EV0yWAG=nc8*mDu9zM@dj&CAtqT`M&$g?Wx?gPBR^L)N`tp$sM zF!O?N{?Vb_+2ORKk+NLE+=Sw&R8(B;JJPnD7353T<+^e;`-o<)EX~g+zi72uC+ybK z^pm1lWD#5GiAi&SoQzaI|TRV2(TFyP} z6qIlEGM6ye$4A(X$sL*Il0)cFk#(BS)%#%0Ly~|Tyu~)jpl_W!a)qCVp4*^`vq)RP zNY#O)WPPW-6HZmF+b)oLI6LL>YGxOYY~chUujeCWj-A?XP2cHV z6H!)Vtmt@bI&mMuw*5YIQ1!BIx3$ps?16;!5o*l0#vQ`tv4R>ei?2zl$NHNwG(%$8 zIsiomFQi@+oAJp>NLan=;JM>$g3uO!&cusEw5|pVuY8PH0TNQJ4|>eIp$pCc$?4*z zPQ`QG7R*z5_|U(3XqCSl2XTQm15!v(&E$n>b$fp*ROX4lE=t|+bQxvlECLRP9CR>O z#qVMGw6~;C?PD@lX+~u>{IBv$yXu@=pWP1cfBcTA|XOlS<2XtS9=c{`3a+3!Mk@KWrpE?KmB zrVAL?*~TQI&GIwne__yBotC>AghQE%@CQr+c_$-*=UqxM{_0vOqwxpYV$I><3^Bg6 zx|g!TmiYsUI2*S@6dAcOB8r2KOTWq=uumMdL$_7)<(C|9#vfTY-ZUTG+q5oqS)`P2 zIL!6EyCH5KbYbj83EtIbx6xD&FP_{LG$PEaS>xd0^KUxifxJ9-op6OAT_OxC{^9Z4EjYG8m(#vBn>!* z5{#%K83`Fk_!Ta(XX~PzD5AusoS3H8MnU6BWwBDD8(H<#{Y+euX~x>Oiy4B*+{fXU zMeq{y8OL~%AAMsD$9TBA7J5hpEcY&6S#Yue8&!Yoq~u zb3NeH5QUlx@1i$2I0m#n9%dh!^($v$AFaG^I>5iVeWn|+?+vhZKzB}h*MnYA5zq{d zp=}Q!ncGXDgSwh5xtmWvt1q9-Q>1;k*0Opp`|u8kc;~* zEcK*0l|8r{GZ}r@#)XnP?VlXyEv4Ca8-l9^yq1xuSLu6Fiy&|$BSxq~6Bb~kuQ*5n zIwKypON{e$RH+@loDaLSMJ-Twca}$UJqI6XqpLODCzWp=b{oXc#`jj1I*nz;!svy0 zGv%^Umkt$8?e8!k*T#9NT_SNPy!ulPAFs8Bv*-pwO>4bg>5;W{QV9z{b%=*)6=Zkd zhf(OECH1YpKR4GX6u)HCy_Vg8I$!n>Yo8z8pQ9OH2?q;Rh>RC+0kN z(PiQ3@}sIG@}sK&B!D|2C1RN>%XP@(N%==~@Pvi$@!wD28v0D@(Boh;{Oe*+bKk2Q zTU(ZErL}R&-enqU?`ku?;QkUhc~D_lZa2I;_Nh<1C6)&%(BM#4l&|veQH#Xth-)E> zEa~_s(L97ORa=K0nWKsN)lJoO$ zk$KB$#TKZ->L3ZBy7ksf3L^J{=Lr=i*5~XU;p1eYYS*xK4#A(p5zB5{-iqC7O1*+R)&FlrF~oO#~{iHZH1GK&!q;yViAi zBtm{n?KNpx;b6a`wjqPC4f;)kjTy!?g{xDm$`L4PDf7%`VSo-8*&WXGG#3>;>57D| zC53B9=spnz1~-WK<24{)@WN1!)Xt8)-d&>KZd;fVq+M$qW|v$lt5}$MDO$E{bx;Zo z5uW#USrkYLy->wo@K8*O>)dmr!Bc>cZF?VV>CtZ2bh~_mUjLdAQv-9((QvJyhCUDg zl!r$_63_u-l}^a+PpMfc#gc_pMcOF{cU!G)3akp|Zjq&1jh0tnvof0qi&rVD%fe6s zggUd>BsVUDGp=2GOnvhfEa&P0R>N=ioLi_N(RCTfpQg!pabhE#(#J20 z^~~`-upw7HyvG;tR5MLMP(@NnStFl`>7xjXEzZUI4|#_j%pwk(k|hmHi9-?&TR5(t z^Fruzatp7p1R?#_YUtpEZ@xW@RZ`d?VfxdVvB&2dc9oRGN!64n9e5`=nIS zTvzc?qs>z}q%}wbv~_2R0~(I=T6Ni+Gt&Ni`>KbfNw`230= zZ~0GA5i+N0IsCy8E&7s&SC0AG#A!1@$ffPJKHfGPM}RZp2yx8?`<)XD%D}`#WkdpX7 z>$WyejP}*shskjH&k4!^w7(Ffg++j?Y`M2-(~fjI4laU0r^DBX9g#klNlT+r<4uCELNe z_TtC@rcLD16!kO(b>-gW3yT85e=ve-V}-9&Z=GrO^O$a#fpn zmU8n%N}X?5r=Os`AtVcyMda}LkFcD77J5^+NzKu{4&65gHd%Of2Ny_7$YVZ>j?ji! z{YAC`$Tp|(Dn0qv$+{qP_o65+n;R413>LC1_X7+D(vdNM0pBAaTe9cm952&{3W4NP zhd=tNjp_D2fYc&PkBXuY!{l>ITo>(LZhV$`t@7}bq{yurbIHaeV*b8wPh7`}P+`4B zZO^ZR5NR!fFQ)Tf!yD0W&z(w4guSF;a05^(}0tXvgCL)XX( zTFouTaQSvy1RJ;D)-1dpIoeL!gP|NCdGIUXwhfhryqv6h5l-wt$;|hb{W`~omgeRS zybZqqSTYdbb+3pHYB^JCXu0NS=1Ks1Gy~_49Ohi}TCEf1Gd2!uAYU25j9>w*+S{m^ zi!7awux|8s8M$2d+xg+`l&<&n*!Y1*`9^#59|=+Yv$|r?lO#|Dm6zmhB!EiLD^Jc) zJon4=VY0Q>F#F5&5kZV~#QXs9P%!DF&;(oq{5Qd;W4V5Kt~)vwFQdVFYN<+N&tW*z z5L$>N!fJtbib2y=nZB+`^)9teDM;n?&#>^diB~^^#F@gSnEzI(k9Xqwu}WQ{@rwv8g0XjY2{)&&MNiUktI5IEsVc9zwtNY1#tf) zevCd-w&miv)+SgfED{dk;KZ8`KZ=jg%+?w|kIynx*(y*3p<1@woDvXjAbDU#Z-$l8 zMJK`OG&)D6@v4EFi&FxG&byq&NqB2*<)xRG+(Y$(@z4?bLO#d&<^LOB3(mmtC7~3E z3lp^R^*NAC3)&fo_t*%m;Qdq^|a-bKU3YXzO0FoWfqdDWP-vRTi*& zEX%Q{xNeKwk5XzCvudX1|r?f=@9{?}>=fJAh&5S<>Uxa9r zbTRzxnl#WJ*MEQ9l7JK@!SBD_t+@FZKO;c;aS!UY|HnUk7;z!XP+6=R zYdQI{XJpLlEQ3geA9Ltr_?3sBHT_|q<$ipr<}dX1Z^pUc;TMoiwduceVBg8& z;@z+Fj`@&bRHQi=B7b|>G`nMb>zO*lyFRWL$z9!J_hke(p2h9-|6pSwaZlA&Z#4Kc z1Azsb!w7)+(HsMj+#uj)F}{d!NUAT6yH_ z!TGVyx?WMF_L+ux)$WdwFAswo&R6=|@%Uf4)Bo+k`5$~f6~ZCifg+>23>POo8+kB6 zhlr5uuf1Zvug+fAD&`s2FM&3|KGeuWCT;SFcsU)_qzTA*F=P2Jkx%G2sPBeP+r)H z)7Wo$s>>+2x8Pyg$yp_m=|!qN@~E_vmPcs{bcp$&zZ*34>M}t^WUd6ierHOqHq!dA zoBR?&#BCzQd9RnaF{-Cz_N&~Mp?v!D!I*SWRE?BTo#1oQdLza9bA7Dwiyhh@uNnw> z`}cMop4ITjb0sPyH8`Vt5zwnogHHT;R*?~IRuRsv`RIV!Ai4W6Faw;3Qj6(aG>XFc zjzbl!c>ivn5XY;Q>hA{&62YclPJivy0Z%P&dd4|pSY(A6xa)4@iV44#*T`xOGmRQ9 z60S(=j^1e@sd<*7$us_NMRf$*NzU%vxU4aiA?!YkYEw*fDbZhxm!yOU$C)_IeBbj< zxFfwdVaM!D_-Z(($Bxm!=!c<+V8l=}!MCH@JW?#|8&hJl_uo{X9+?IuJZL}3m29)7 z5%BoHPO~i+p!V1o^#!|r%!?2QsZSanb>TV8Mqfiu$9)%pe9*a@Gw}@cZq-~Wt3Obd z^X>hO^Rq{mt0E1#*GITv7X#5ZFTZjQ|1i`*@al3_A5ogNHTiuS{2RUcZziB*wE>~m z$pysDMzLzXeyx^Ha^}QQ?DzV`PS(n5UnqCU)Y6V;>r%&3M|FKke2eAG+DeyZPAG{h z<4|Y7B>oMcU>%oKgDlKC5UhkIuCi=73Hmn3DicDD-514}j$jKP-t6o;Bg05sS3TUz z7cvqo?KnrNK!kp<^NIx4S*t}%V8Ie`{q}Z}uc&Vh6UlZebYQaQA@@c1oZvg&3um-~ zv$JGhBN`0L6w`(Z-PN|j52)1oupGb z%4j=~Ah;_27e8;Fofu*mQW)H&c2CSn^SR&X{zk`^@0^((>K1i%eIn}sBk%Y8nayx5 z&%K^IHxq^Zy-@dzSKUgQ;w%~a7BC@IO04*mXvFIMD1rD&u}T|B`mV&G)aLB|X?^}L zHHV0%*!gjK6h(akXQ}?`m3v+9@3^qf#0chlJzHTW7<=Iqru9Td9{VHZN21WeEOY*k z*O8benI=(}KJG%jo(5)}i7G_%jWN;gNAn zys$0nz?WF6>NnL^GLhMLAC@Udc}#t5+ua#4kF>o*JPf}cqAyrrssuO5sg+JQysndy z$WzD7qAOu;q5PKbL==jOVZ4O)OJ)7`<)@xm3+*g_`K^c(1P1sB67HKSM}B6a$YiHS zaju5?2}fr=4SX%CS~!(3#SrvaA(vaU&7VC#I%um|l3jSeoFA)Uw0Ffanf$tLrQmZ( z=Fe*D(`}2YvzNr=0zJmPH3EfP?Gs+J2>K%%Zt7giaBOGn={&Tqt|T0tb*sMq#)Gwucf*@Xt~=eXi{UHZ z#}=GwH%e-o!}p3zvUVqa&|A2rhSbaX5Z(WaA7)R2iBn70CRTi(a|^s$FO!q`(&%}( zwc&85)HF6H6w$yOl=9K=gSM8H2-OXw#V0sv`HzyjomR$lNmj;TTcsbXp2t7_5KZ}S z3pBVYLfUJ_9`+p&_ZUwX*Z^q4w#;Xm4|ZUrB2Luq4ETM zh;an|Rw`_TwP9}ZvN$31p+-%gF2xcLRAp-$eo3*p<*dtmt)1)Hv?lNcHuilvrqn1AQy=sT~4t)XNdYOl&d(a;sc%mrSv z*03CVt?c4Q#*RZ!%C{uM@qUU;%(zu*uA22-4q9JsW4@iFVOK;S z%fjv7STAj|__iyjGu~I!b|Pl2FE{^{o?zLj# znCsElkjQGP^rGCrma&~14)G1Lgh};&GkeB2xcS7wxL%(Snp#(%bLWFm?JJJiBb6sQ z-gfVLhz1K@EAMPv#6R~}Klk~FY8Zxc%LAqTIh9S0wqNc#m>1ALm~g1W3!A!r8o}SQ zduquYp7UCb-#=$D)p7hwx>L)OL-Zw)10B^g`4}3$!lQs_lI7hHaSuPgB_-j>sjlQ=dDUQKoj z^YKJ?12Y{Bx62cTA@dzcZh_Ay9lg&_$~QI~KB(Njt~KLju3;orOptODu7xG9A0tB4 z5BF!1Y%g38$hKQM(ic7ht&gXv(jzNYbiA zVYmY-sAEa=iShpBehbE-iq59;ey`5MD3recgD{%rd6b*Uc%RU#(7Nmq_8MV*&xB$k ziY3+?7REc`>*CavfeQ`{*|^gb-|Ze^2Um8)jZ-syMGI?O;33`+7wxZ5&#L)*gw<}f zBKj`KE?$A?Wik~|-Gz3NU_!a3EKH#6N76&P8xqC)C&7+*#9mvWVRnw)B-Dz2NI_#M z^!YWWA0tstI=6OZ`kBwaB}iT#$j*}z{9yEH>=2QKc}l^01$xNyarD$ufda{j|TsCOR4bzWoIzbYu;@0oRQs}n}|jG|yLK4}HpE_30r^Prw4tlaY+ zHCMk{RE(z{?Z)EUja$XVM_yCjI*?l5oCt{I=m)8)4)@2-2oKN7q=X#hn7iCL<)Ioz zS!N`zR5b5=u@G8HNiex`MJpVm3UQ-)?1uI{!hPvP0v?P*gzfq}!@E1`|LIvsO0h(y zF%BME_l4kRnMhh9;(#nw3c~#>cj2E#J7;pMvJ@;I&o8M)y}TbcH2qzH5PH8@`eXA2L#*$ulV$W``>+{;ggMOns zPDFwQ_b!>1KfnL>YIaVlufZb~h?VhCSd*U$n50>)nFa$@lxXaNDC)Q|^9=#VO~b~3 zm%t&m&*0MP`g{fp#nQyqhF!m+Fk(ojoJ}0N#4tbRhnfFh?7e4Hlx?yutfC-_A_5X5 z2neX;tfZDKlCyw-WXU2)QX?WDIZ4hrXOK)2B4#nc4Gt&YT^- z^{sXO(W`&-!~N7P}D%({h!OH-&T$foDlM{Vmi#`L8&!k*-p__ zkIFXVrC)G33z{6tgz(~P=kno2mk=td2N;O#=f{qpjC?1Km0r2V_?!F2|MMyW2cfrf z*U%KFZSh)H(~Ej*^oAJXTvyQE#ooAQ=pz9^9L_SStkCE*t#Y`ov|J1)u^~LLWl=7I zt9qf+6ZYEFWIUl)<32t=#@1l+n!G+8&W^A`X9a}o+iVFPhkz+NV<5CK0aT1lE^rDkW_@b$Cv zNH@A4S-$~2xMc2}>!l)e^Ixi6CgF9GG|ru6LB0oV5Il0%Ap`@QLr~+A8gqi^a0{#l ziBwh&u&Xpu1Vm5O z)qrzU;J;BT`KPh!B8IuwP+c4cFY_&gD)pA&DjnJ1y4YXHj_4Ko?HgQ1s~O`(RA!Cv z?Bb9Nq`84Pm7<>!cJu?^G1qdE_WiTFE~sIa#`d!6k4O_*iBBHMP;BvtN6ha^3S=>y zQ$7%U2>Za(5O#K=ob}{wBjRzxHccKYD(YJIO|Ds9d5D$mzEvs|X|}{9{40x$ z)nJ>l9bz`J4$oTfey!PW9o}WEt8su+@ZJHr zg~L|ubxQUd9ig;5nN@c1dq&-H{^21~J=MoIRC1LifR6MI0fB*i6%tNyycICv{ zSrayS`-{6@@l+nF>DgG=UstlW8%kkTE8r*QFlD#jJiA8eu)Q=CRf)Z-6nB}@A9;Gw#%?gwhsvdZp)5L)dEvz&Yu6emD?d1G~J z_2Vxv_h#Q34>-{+B8-~2sNbNdUrE}U=#Sy-FznlH7p_!Z2LSz*eRU3^2d>rr<8{XL z+?KCB)U+K5`ygs}E3D?k{fQw#B69U^TdL|jAgVZ=@vmpZ@3l`T-A#gSL2qaM0{G&TM3?E| z8&I~QHbNZuZvgeLaBA`EeQtNCVIHpZXe7T}W+mgby+qDi&Yo#9_Kl;nPWy6O5Tytl zHzwv_w5gJ9D40e0bA-NH{ox?jubqd$9u^Vf@EWb$Lf0mMx z63lM+@+n&pmLKD`P$N~P^`efbgai#}r>_JHeF7=aJ;WMl*)TdLc7yh$g`j~uZ9d)4 z-L|GlURS=m+34t1RIayHxQyfO^pM$-HYh%y?D9x^E(ek_2W6ut3 zBDF`~W+#S!sp-v5tl2LSmqO=>o;u57)VQ5gTiDjSSt>Ca)m~&b&eE>6@!-fRUQOJK zy>#H$^Ie8(xKGBkB`_S>V$?+1&{55M3*n;pzd5uA6jx;LP3A`;I~y#C`)iJl@9;UG z&1GqBt@z!`Q(R3Rnd5}#doD|j(MX0qyq>%BjrHpsdn}k2o|TCI z5_y;mMVc-?WZzi*9J-C{Z`k6Cz-@Zraz34(@(aGaY| zJ*sRwJg>O7Y&lsUq-~6LajPI3czj`xAhS;Ifc_rE_ym3o^R#Cm09TKwTa&6JDf4-G z(T&l^tY)pk2HjEETxM$bS`oR$lS*kXlBKuY&QADd%|;%oqH@Ax-+@M<;Q~!2am_RO z8$X70^{GnZ# zx%tEcsirB|RwRcNZH+_mlnhJ-!;hY}`thUD^+h5TO^Knav=&3p1{&+I>6^t|3_8)R z;HuLB*+;ATnNN)($Q-1aHgfZDVh%>MdiKnlD8UVH@D$?m{ek(}uFalP%@v;c zzxTW^w(U1ED+KKa6d-)U^%mQL2bb#So?Zr&P0%0Syiq{rIeH6w&J`AgoN>V3dVJ}Z zH?N8Vj3EE}!>@o5tVzU(KM2D-BwIMwq`{B4Lq+A=?$%w&^`3FsG;~ZI_Ia(LzMl4n zfmS@G{T%!1+uy$e+&jPU_0&PQyX*39K<9PsQhP!;0rA=TEB=teOHKz{pAw-+HdW`H zg}xN4ZL0B7WBO|7G@B}_gw_m?fWSW|d0D@#C!zd0$)NG~eqD629Xt|TW1~;E5a)3a z?~I;}n;1yg$~Z%Wg?#j2jotPmy~*^AQllO(Fcc*lXPw;!d9rt&B&I|S4rg?>3D$uPQn2O(lhl{@xrUJ56ml~-ST4{CzywnUUqObKcr($ zZwIbr)3f9>E6R<`LT*{=ROmlgcRgGLqh|Y+;B}fEv`vNYFYaS+Rq}7CpQKW#J8Mzb z`Mqgs9K;!C2Wk&{Kpo;X%hPr!`-i8PB`FF2ZMO0N5C0+sY+1aC;rveiPQ=Zjqvb9l zF}i(LIKdja;{iXadyg0_co?;Mg`W#bL4*UX(VmA8w;Uw!y=`YC{FlsnwGNocEfS*z(8Cw=(im4D^ewloXTGCmYgV7s| z9@Pj`VmnkU?NuY z09xR%FQOMYA<;*1M*W8gx(%#W{@w%&=zpb{VfEnf*&8`b`mXt!VxG`;}Iwfz*fk*a_f3`;x#ry2aMT}0R5X5rlbFAj+6tbC)E;@VMx0-Szb$L(!4`8`P!8@r>)OuQ@V$nwxPUca38aVIX*@GS7YV zBVtL?_Sk4O+^8_uEH)Qletn8{i*!`k&dd7r=e)-_^1!InE!C?hU#1KY-_3r*>yMp(*-+p3G*AdB(jvjG+;c@mQ#H0A63mks{ zT-36@gGS#=8k@VWjf(=lg?o9!!L1Tys=Bet5PD=x_BWq;TTPy=X`YtY(UmNHARVZ% z)F1ge+RN;pNp#?c0AJuRTrsl2WEj0|@ymoM2R?$ah5g1xorn@?k2k z;$8%ky5ElMFV-ZjT@HC-_(VFESB<36JBZl8h?+%hiz$TZvB$>ZNvaUqVPrj~Jmjf8qFu}I z4PgJW+mGunv!hPh1bB7(60}wb^u^`GV}?xzGIbmXSEX1YgewXXvz!a&iZB8GRi4|> zEgz^<;7~d2Zm0*|Lw6AOp7>c*D^hn}KLo71Dl?rSa7#Ru(gM3Mb#ch~Uk0e>$`W#5 z{%?Uoru$3<|BN8%9K?Yk>sQsnEx~rE*KrB)7_jyZ5G-Ce{Vha#PMN~_^>&@F%WgL< zeaQUsLsHRPo;qzK>JZrMQ>>$xe?&2g>xsaxdioLUWnE_0ei~$Pj-miFLi`o_@VBp) z7=tQDn{4wr^q2yTwZA(Xo&Tpc{zLDI=MBQf`%R}cHmh>Srd>o|0t4+`c-DuEpRX$x z<}gmx93j4s>A3kYD6m0h)T3z{F_p?SL}WTQevM)0``uZt?t+ia}B=UZxfI0q4M-nUO`?NF7e z8>^dbY1kYgOq%w!Q+dp?iGs`d3=3Ti%q%$?>AW#qt~zy;Sh(a3KKQW3^~?kzFP!Li z2c`t3W48z`9JUOuLMJR$yIMyP_e6jyo=KI_jeQNP0aa=K(N(?h>P zR^4e^gT4fefVupDQschwV=_;QZ0MZ_4;0u^rWZAbeAoeqEgA_9PXqaPGr_@Kzk&PQ9SOcRt{?kOz^!WsU!xpEQQ z(2Cp4+Od)#Ybi%nK&#P@ozDLr*Mv!Jc0K^I}ztoKa!`DXh)J>*WT+cTFGf$evato25 zzuFm&<#r0~?ylaNpzAUj${>V~S-Z(78|n9*eUK#+K*k=Fd`#?O!CmT(zK=TGR8b#Y zy!pJ|Sj>;#lSTqJ7e(67Le}C-$a4#kDj_3o*_bAj>hSLD6tZ(G)&FP74rsgBIoug- zka_wyyxYcVK;W6hu$)^!t=J26?ZA%f{ac87Fb?))z`EPC-?_nl;-d-=0jy7)fDPSfroR9x z|J65b*CtiK^-Hwkkc`j=)mi-LFdIyhWDul`y`YOqZ{xvV?*`pXjCKt^w-(BaHM;gE z1BaN0PS+u%u(p=2~=i|eD=l3nEYFpAmV#i~waU zWsF9o8soZMFqnYN!g963JOoZURiW7^cw%CCRXRY#@=-_L%Z{*2d6mI0f)=f5uSh6=EUx0IQ+B>i&&`?Pd0t)sB0Ufig{$IB`D-`cXz1SL2$z9?DUy*BHOiC*lG7# z0TgmD%n_VuRBmJ&nkZ^HUQ{*tJdAvK)dV(vNW|5yP?a*c`7(23**FYx zzQEy6!pHBmuFb7+(iKoB8P+ym`czQvzW&`m7gIC{+#!b8#~feLm9^d8p|amXPLMr* z_)wI%OD%V=e}k~w$!s2dG@0BGc5NWnfrw+M8i9XL6<>z5ydfe#7EjRp;{p5LYwhzx z&G4Pv_USW)vBMngm>vliUJCj%BtFxenZaSW=V<=qe9w(=yed-?u!xZ|wP!EQxkB)f z`mSNfcl~T)SdK;)pY;N+VQ3^eu3$!z!xb!*S15XpXIK(B*2y?&+#7cfo=rPYq2q8` zMdPF6{gpImGIzxy}okudQ6I=JLQcFu5oCv!IE|%)+X0|h9`A0s7dDO-%B(8 zPLWp+s_iu_Ce?0?z`{o5{|_y!Ai%v&I(6ZZA~#RymjK@y0n3}I2U|T7KS%Nsw@j~F zVcIY5l_$CtU#5}p%-!L08z%>i{3;{^xYvh1?E*%mR&kAx%Nc$l7YQgbE-#!`_i>eYIMR z3^K3V@SuE6nDW8Q23x=$Mu0R*7k1$ z38#!l_A?pl0+i8tj6lj#^VzypO@!Or(-9Ayt8zsKCa^2C* z#g=GO1&G{82cr}_pHf8-pI+armJ!TaKjU#K)I)qb>(#9@9?NM0OeFHD^P&HSX8B9v z7%mG>nB1A?3KMA-@7bS8>>ga!!T<8Q7;mgVOO4JBKHlsd(1I^VAU`qUhFq60v#DKO z?yU2%r*z30B{Lh#H!WCHXL&nRE%4-P;paL%be>ejZ6lk%#OO}*dh-+NQtf*s{1+LH zPeVtTTdKMrl5LV7aAq~qp9jhuUCT4xmhoxTFeoh0?jKhcR)2GE&w$cy_>wjqcc$|f z12)%yFyTOUj3D6n^kC}3&Mf~6R0Qz{H@~{0$ImRjTzANvQ!qx`LboXQ$K&#H&45Nz zfCG`hUa0B$HF~^_OWP>_n*cz_C5wQYRXHEd9*P;;oG8WD@+{V%V2Z5}Pk)&l#c9H* zTBxOa7;??zUS!4&V7W)=hSGdWmF<&Tw@2~B;X_6`G`YwbfCX4)9&mRQq{!Bpu0)l{ zC%cH1nao_Cb3Nk1yuHPQGcQ?eyTcpa;7x zSYvfUWLDa)-=>5gu%m_sTg*ho55r77y591zk4x!0y6Hwi>`%W87@YN%*x8N-N8Bz2%aSRvm@B{!UkQa+{3n}69!P` z#py%9OKcM5zLQRJ^wAVJE|bo8F5h1an273TE$aJ&aAL~|2`=YrenZLTfO(HO#w?KP zrg#dDTxxY*qJT}vv7RCFT+WvYrU8;wC1rB-X)~||uB-ngwg6<*XB~jNN3BgX>s$p! zFx0RmG&j@$g^$V@o<<~=hPPSjYDcO!1~eakE#!x}qI$i22aH}1B`Yyp)I+->+Vd8^ z?AbJxFg<%a%9WmVtVt{#WNZ{>gIm_50ift)Q|O zpc}$%!7H7}t7cKu@iMv!I<}R_W;uO54u2wUNdcy^FkJtG*wlnW=&R2GWyx#^{0rIQ2V#Q*Uh)LFt!kGQ3}Ee>_@4 zmuuJb3~#8XkWd5FvG8N>fKZRFuJ~{@FY5LD z)GD3TV8uazt*6ZyGVq?2&v`Qe6kOOCpl3WUe zy#K%|GT|T$NeLwe#`ci@%>{k9i@E9$+7JBA4T6rHPxCbDUTawy+zj>F#0)Tv(GMgM zAgOx4(l}vbY`wGKSUOwE`r!p6?&*%V90)q8HQfm4OX5C&m$9M^T>wF0S`D&9XIZld zH?$oKh!}1VHl*d*!bvEau_MgTBQtV;Nked^J%6uuBv+;j4_ zb~z$=%st2LT0bzomg26J)8O$Zcl$yA)s^m0l4cIyLRQr;2j$HqLo4(1-jqJ2J*37N z(zyDApQE4bG=s8oL%YKKDDz4cr&C{I&1*xQNHeGR7JYkOlr90@M-$+dV8Q-Bn zJWhMbM33bH$F?hjC6UvCfvo_?Zgti&XZm+FP~5#sE!c4%`^0%Lt@u@)y{B(4>$ko* zu4__}O#7reUps7mzKzKHP@%3BC+OOVhy?^OV-!r$;2q4u=#AUFC(%HuGvsuq&AN6D z36p-&x)KJuR6Ilr#=pSF3K!!w!h6`_@7=h^_^G}Q>oJY<5e22*YC+;iIuI`>=ywH_ zLtVWVcG_&dw|}dbpO$;^f*sKPL#57nB})_4D$ccM8rL{V6#;!eY#a)#iAQi_QgD==C5PGbZ7gA?2+NSa%nw>U7#^?FQun7IWe)N866zmN*nE6HyGM|T_=pa10HTiA*Ggty5l{eMX z^vTnP-)^{1IkEN$#l_}wA3;aVV3l_-(9~LZY)8YXD$$ z3k#KME^myQd9`i`&pD(+;;C04dx5bs8Hic4cSw z?*711+JCe@rDdTfaAc0{GHCX3{CcfBz%uvLf3jrNi_>KfWy<-wvC8+=CS3bdP1sM` z%QW+!Idh~+D_>`ZN*o6(Jd#ghtw&ojVBXqF-C#F{UbRxq`od+V8Y4;c>C-1xm524g z(UV7;bAG*)V-J1bILX8`KzM}|ut6ixvv#=KqNwMwvmF_qJ6&0dK@-On6 zJHo&bXLCuU!VVlQ(b=<2(x0n?Ty4rUX>`hKjcwb#%FAeenXr zxLSybj@Q5cTLSZ+RAR>;{8KOi5yYrLP;>ZY5|9i<7QXUd=b+2Z_0JOV0#yLhbOO====&~GoB}ZYPp*G#(bihWWE_Q z>m*kdTck82U->m)-rIm+=|{F+-vj+t>0mZs;ytGiOB;18?`};V`1y(%g@m#NB(mNG zlYUh7cmG_BckI!S@R=+ymfv##EN8Ki zLt6v6pe^SOe)SjSUC5~gw*>FkZTAKBzNr5t$B&DaZZa<-SVAUAV^WQ^_xhCA?ya8V z!JQnxJ1TZZJtOIgU@3Wxw+CvbHHh%w@*2FvK#6*nlNO?fh1ecvg~8G5hUUA$#h_)5 zQN83k&G~CG{z_bOh<5SZ(#Xl{5da%ti%$eVHGp$40D20?it+0CbSmBR4OU_y2M+N$T z)5QI)44h5z_-^E|+D{hcwa12^BM%f40a7*RS@383O>(mp@lpEKK&mr@fPiF7L|9l^ z(M!KANUu9ejw+0USfVfi-9khzrPn40$>L&GXDQ9@V!7SwaJtv`jMMzJ5?5DoB(pY6 zr=;@XL0INl;5NI;xsm5f>IqS;@aH0`?S(Ha(MYNA{YJQFO0ij__J@=7L1Hc=Vr~l! z%G{cvay}B9pg&r-wiCB*I0nh_pzoT#i9}b32dap_+;%|z zDBsAK@sJpYIf$Z9yT;~H;dvKok`4IZ#6*)fM|iF#@^T^k*KSkrKIq;Y*a>_BIY3L%u3)jvpXT+Qu{Dg)aJbLov z$t)CvOrZ_sY!%E8eX@g`%%v~UjthOS=nf2}MsH%A+#1f)n%E0{Wf(9P!C)@nhT0mAIsm%1e7!pODoLMcyrs& z#w1uSnwl1Yv+ckU2@bqG;nmXPi@Hwr`5C&Yg^0*67V~f*^?M|x&9c}9e`3C~)3RVI z!&VR9PM^Jq4KDv$qPe@>Md@8Z<2<{nY@`!CKQ={$vckhW1SR%5JLd;LYQ5$D8POJ- z0)>9Q|1sI_qN;?lbduG;G7}+v%D?ys{ibng zO$-pII^PVK3bbmxst=m?Zw%9rg0>in8I_RVfa*A?*W(RT0RC_qZ5J%)a?kFZ(xu_; zgr74Y}jl=niN*km21CBGzP3?0O zqoCnS6%)TfmB}m?9nsG`UkYPK4|w63T$Nb-Pk)I$=T^Oc2(cB-8T-d2{7G7J(am%1 zY96LN(Px~Co*q<^`^?MlGpo~|FZx%1C4B$Zk9Q?AQP)?4JvqBGSQ^9m$2go9e-5G* zZhbHEm-0-SXF}=^>E3W3y6%Q)Bri*LhK3|vq4s^*`y%+^6B1GE z6yiJ0^R7=aINU#879tQTDM?Bz%q?$=o{e5lm|365nK6b!?5l;<>JCn8oV+*AhgTR> zsu65Hd%SfnjuZ8+nsKBAhTaW7gi@Ejv(vxymbEAu!dVr*bmh*+OIIIUx_tfl-~ZX4 z(4b{8*CObRyX`?kaKo zC8XG@-b-d=bc*u+(*YJ}OnFopBDdep-7N?fhf%gs}2OU+%Tce0VtjTVdV{lYV490dA-o&nHj zUV@LwfkW9-XR1l;({s;Jg2fK!?d>oD1ffAd$7fxE;X43WNO#_jE^b`r!vRGmI~ z^Wjha(~JK<*zgMU-sDVx{u>8;HohK_r~4eSA14Ml&inh!uNX=6ol3<+10UbOaNdZ5 z&*JZa`2-6W^9j`1iek-E{0n7I6=%x)D4EhNudJ=o>8WZMqvQHaiO2PIlz8Dpe5Nly z(fu&iBFAvCtzS1JpI206($l)84yr>|XTP`$93G^WLW@%~ z9=^Q-9UTFh`1~uIB7gs`|JDLN7GFJOEUUrtcAt^?j^MB}B$1Qsx>(`L3gu7OT?^#EZWG;0w`O_68wR6=L@W{x;AB zoaA|d$KlHOZ+ILWM7Dl`vs^dm`tzj(U4Oc7f(q3>WLr*pwzU@XT;AU^O0&5Ye_$h+ z3%+r^2CgO|>oIoB$3#0j9E4ErbJTBa;|=3w$j!~IM%{nSHi*X;;e^^CL7XqPM#x-3 z*OIKi2Bg7X>_X^&z%D2;509NZBR4X%KA4j^<3ER3iSv&21G6Y4#XKBW3+_d>1jd9W zVIQljwKY(b-6+HT4_@HkEu;1B&(i)a3NlQ49=y^a{?C6k2i(7yC~SCrH7bR*C^N8X zOMva%R_zfKv~_y}qtu>HuQnl~xZgH=wZAVmq?hZLDR3JbNKKSM?FQLKv*%8X z@{z1NpzuPs+s(v=Qdh(M-cZ21a_u+Ibf@2^0qnlGd%F$Xni>ThgeA36rsH*Qs1ZIZ zpVJB@GFqkqu@>!?J=Di0ru#;Y+-AmE572k=r(lo^E{3y9U6YjE5Hmk_U>$Z+A`eZx z8OYp$Q+b5p?5ABSc97h?Lp*L*|I&v^;o|OKuocmhK02WKkIwSnkBRr!F$LbosEPeD zVG>W75+lRSHIS-f`V?#ACi0pXH`iv$zkjf)`Q=oMxE;B?PmI6qUL4rW`9WH&{TWSW z2K7g%kq*$)t-7c-H&CERhty%?3QBi5FGJuP#N!^S?uWm5l!jNoJeo&y!xo*MeT?_J zHDm6}rK!ee$%zwJy8{zWB6-ndN(7jP!)#-9^eX_&wOoKX)}ppChFWEw%_jZD@z_g9 zM?h|+Nk8$JQkKzt%#K%NW`=cMMMKwNA z@henhe|;Z+eUx-rNE?BtyMSG{-L}sOXi?n(Evhh49E9D5Ls<1eO=reIt{4f|-Xw^y zyQJrcKq7RB6svLH*k5gU?92av4M!4U)R4~q*nhHv6?5~#UWlk~+rC&MdNkW`|KFY@ z(*1wnN&fscT{u?4wEs^X>otovYRD+$7>n(w+$vnKqHX0@#IuCmAg26H$TNt(FcOEw ze<|dp=^Ue%7vQ`dA`U``LYg3jrk&9Bk~2H zVuZCU=IUj5ZOxZW7+GE=eUy%8ml}Q=HwN~2E;ZxMfjxUDKK0}y2}%;`^8#z#NBOsJ zleGskAEkA%z|4c%#~xd^k`Nj;RhsO!L?(84ptqz<-|~N!deZxq#|!#cB4POU0892$ zrCDNRI+>#)pC~MC!G9|10^v~*DJ48HIE!YgkVSgV>97_#>!|UXtdcWvO@k;Z^F6N& zIOQWn97J*90T5s3U#7 zJ`1($i|+q4S^5?9+_SEJb*y_kkP;eV{rpfPw)Wxf%LsP?x3eoHYgLhHZyGt*9d(RP zR$?KPZwZ+cl~;Pz8L#EdjG*>6sm#WpBwV$neS#8RxUR2nQ}92^pX4m|U&lf?m78im zL{m~Qf<=j;;xd+@sS-hxC#6QrUlnPkx@OjUXzVo;Pr6GLfg;t!#zAT7I%PQrgVN@q zc7yLhe=-EY&7fAGcVPVr%*(F%q9pq?x$^51m6zOBvjicOObW@uv=ilK-iVnRp5e1? z7-hGz-Nq8h`$?1Kr9h(trXF_cPN~QzG$x*d22X?D?cAFi)PeoZ*&my^>v759Wf zCD+w)(fh+KR?2IuK8Piz z9pC22n;k0IpliWwhg)zAOhEz#hvWX@_lFa!oj;O;+ELr*I-=?yQgr6pmKOY2!igf7 zLSrrStnP2SAx->j*M?gQMvH{-O@@oI30Sn7WEmDu?^_iF`^fp1<7`U^%RF8@OLC`K z%F7waj zjFm*|P_e6dVNf5^an1c0rVX+K;6tbwuraQ$aUS6`q-%ZQg#&DfdJm)s+iTf)1ny}h z;~qDH?SBIyFI}k$cwT~o;F+<_wKaDOVsvv{hPC!@29QXq)Pejw_Cx` zq}pWG2;y;-j7eIp=-x4Cxp+b}n@mCNf@xKSv5vFxp-e0+5`zQ!?!2 zac`A#b+Zqu<*Lq1cA$e@&^C3?o4wUO77wre$dTN+TO$tkU(aCLs@=6nWMr=yBKbr>`sFAG-e*^x5&&l+v*d+{Sd4u0u@g0wg^PSnM(kZCRNHDyV_{1-gpJ+gHq9n6@-^9jX z>M)VEPRs8LJ3Sk^vIka{g>#wJ7wyRj9B@W+ z?mTw(wk@|oCJ59lr}&w6iQwKY>a&>0@j+DhZ*7c#P9l?;oP1XVcM+e_>CaQpP5$y= z1FQmGUFk$T>lVLBU-@alw_4TpxLdbj7OSiOww6-b{w;OU5~ z>T55(x#~43k9Bi}DLh5Y&v4S+b+p@4G3A4#Nq<&C-C}jlRJ|5a)$$7F?%ub2_P71b zDfX^?8-CvJ{)SAsf-ZR}8?@Iihf7{UdJSn9{(4~*vih~Hmzw(AeO8n&W}bY6XXkpba)mQoQW z%d4J{ASeet)-AQ&a&iEEGzzrjoUB6RBEd55PKP$Vq772{X{z35_quoUO(pLd{dR{&DbvA1 zcYDnZiJgTO3&A#WhuV0sy4}lVr^R?Q@-juFR*i$##>ulERB@Cqk)uKo*YX{F666VJ z$|m0*Qo9ccNlv7E*^%wxF7S-ZMu`Cc%8_A72oAzWON!Sd^3~F?DgapsI3Ue0fR`pn zAIJr`K2d)JQw>YXZvKvu_#*T`9rKW(uu@}db3#q<_@@BL+f8bUv+A{!$=!Yr41vlD_ek6_yO17U9A=P>hzp&!coL(e~(fBp!a_!#*k`@B9Qf4H_fznjd-)71!G zZ`CDX#ZOCF=<=j7;;bML>Q>6Sa~&BqSv7B}$QavEWFJDkNR~+HxCl z*`2FWeI>DRK|L7`wtN2*kE)OLf8!JiViLJ*lD|Gt*SN=|{eZ(rw!+y~dgY0=gOklf zm5q12TMrix=t6S)Y){0Zv?A*0gyf&{hl0yn0&V84ncb9$k?OkV(6uE{lcoZPIEbrV zTkec!#&4Ag|8#9Ag|vM$Z1EJFSYILlL|UgYSu=fRr$l#}L)VO>-W0qW=)3m4=pmAX zE{fsxzRhK9Rv&kP=apXEJ*OPXm9VDK|1$3ojd{46nMj4-3VD38yBs!CnLyg=)Rx`*{G6APy?i>b= z69IsNGJ58w4u*qP6I>+&A!i?+yq+=DV4h2o2ur%cTP7Dg+?TG?P6{)d{?sp5jz`$>mO!rK36APUL}(Cwf7b8~kFOU_c@Sl%tLk$JhSq^BK_idXFoj&vl=0}xo=T;@1?7DI>{LBRI z>^5?->|L@BYva6S@XF3L;5aX63j!vMZsxgS{Tlw7cLe}Uyyx3I{jlR1-m5!&0Yi$(@J<=T)AbH0{Zs|+y{gIaxP_H2C5)C;sG>B|smF=0bfhkXf(O*4|oiqE?j zIr7ra9Yv6KO=T6{0DJZrVQ1BHy}agY8EDuco>(Qes z3GACN)GQyn`3D5s(^W6n-W}#RFUPhu;{>C5yi}d1xAp=D`uKaAXvAUfNE<0?ZXp8r z1PswC@ic%ZZYi*XcSG`*cW`9s(RaCuj$-SDW_8%qnd=L}TyYh3GCHJKfX|hh4&zeY zm%7&xXCm=O+{6FZ;@$+NeP%4a>?6xqScL^@qQm!toD_q>CaZSCH=@nSzpEyQWlvvFPl9+F@>S%3pLegM) z+UNp*N*a18Mh(QW zs_Gu#D(nDPaeDR%q{!8Sl6>25v*+nPLP0$#-EI^Yqvog_kn8_ORWR&1g;0AM0)5AB z8>QP-LWa#hucHI9qKryDOP=bnMp>4D0ojZ3?}6@G_esL;UI%2?C$=_HAsCrR;@7|d zmdC_4d3gI>Z2OYy#OTmZ18Ow$m`Zk6Fzt25-`mlGx4-+BbFDeDOaBL2_ZQxm{43rM z1$I$)x?a*Ysf)SiB+owZyTobj#q;QKDZp2m&z3z){s~gC&r+yP{?9|&DV3K~bYAmM z5~U;_-fg)BF4Mw5{T;L-y~4#S-)J+~AS6dR;tkf2T@3VN;EmtE34a+jp6B}<_cNem_t*=64+*$G9)K5j`;mh4%^WZ(BK2BXL}mh3yp&LBI( z%>1sQ`<(ln&&fI8-}n3d>-*=uJd*2uU9ao)dT!V6VYWlZE%sP)i1Pt_AK%d94A*sI z{`I}tebyil;W|G~hup(bcZc+D=Ap=lhJ3tSuR?>eK?;7r~x`aq+=z4gdj|D zUduXLqv$*Mm%0@b$*%!ZyzZVX7b$k0kLmotczZnc0qB&yz#e0~4Di_Ba&KQwrDSM9 zIn{D!zO}u3KQcK3!nh3EXUPZWYg^?}3dk{d?!4Z#J#s3pHTFe-;(kdM{JSLY%L!E0 z4#XGyxxSmn4~sqg-&rW*$D8F@~7>1uO9p7SV3`oDFYz zSu(==8<~%I2cy_o4IR{lCC!4`gw^_kYwFw~fl$H1iRSD+wa)V8ZzB6| zx*s+NepsAHPs&>DuE2xzndfulUwek(Yg4x@TKiTAE1-zJni=2sBVXw|bf?hRuf2{T zpalqdTeyq4adurtg}@O<7Pa7ZkH#e(i+F4-G9UR}B`r>BUpd2d=B}c|@0Bz67TNZW z+mi#18xS4LX~%TXuC|>^PN?5Q5fiP?@n~@^Jo)+%@$~QveN4a!j(w>1)>q$12!%%-VXFxR zDh;-d@|XMY^NW93I@^xp=eX4$o}|CQ3sP7Vbz^QH`7MG~O|y5x;Ri@IFrabj7M#W z1zfMD#%UQo572k)c8sdq7_`GQnWH?|mU0nVpvgz3$NJYEul=zL@NSd&9@a@H$wloB zDeqdov|xronJV2y0oQ~H;C3@>bJ0C7E~K0|bNuMy%oRs_<-64+O1SAYTRkR zitW~20X?U^e&#XQk2xvO*ZdksFyTCnuK;ike_p2FKAAG+mCydX>A7|*Z)q4UbDCA07nnvupL|t%UsQBI{+3;TfsTk{ z86>O~ODTW8lIdHB7ER5&t+wu5#2k19UPr zMRC-(&IJ#}Vn4JiH$1OrwwrcbYL}=Q+_Fg%&1Es^t!8NzzJ6WNs=HiUcf=(!jK`@w z;KgQ~K}U{<{5D~8wD>+#TXl}c$0HQKJAf0yd##enfA@P?lgktkzue4&+*|1*AH&c8 z)A6d?$xVU$w1n^rn-GxYNOy}HL zm|Q8>l8xf$%19y%06>yS*i~vEWpvi$fd|c~m)KDCqWTT(j~_wQp1hLCtO;a(QY)fy z;@bGac-FH|sq>X9OAPc)ZG$CEoox!?YbD5}Cq8lo#`Sz1`Kty>rK;mEqdUqq%7^XZ z@B2(tdr_=ll%WN!QJikPW^>6;-0TN0&h&QG&?aVj_ z$eiYP8lfhWbQ!uclQYkW+oTBa#&y$F%P7`uiG(8KN|{KQ3tqeBfdWC{|zBh3@xTSLsDn{++1vDDtIlXCs&DKXoXb%Fvp_L0KkC4?>9ovr^!fy!SamA z*F!+QniFSyY~#FBNif9L&WMH-by3Z9Ib<3)emUX{VaTDiFbJS~I+~t8bHZ+D=-#N> zQUy}3mL@3Yup2yNscNS#C$V|9Go^VqQ(*#N%wdAo;*l8|Z6yIcaE^Hy_$i!~uah~~ zGgE)kT4&JYj^bz|nXvYZfDA91)x6G~s50EVYtWQ+u}v;r)f896Z<1+%X-n5M$ocd` zSq)#lZk?>PWZM%sRh;FVDccDlPQ)u||HOg7kQKt^8}ZTxln83vT-dfle6g=hBs02Ljo61#n5j1* z;2kxYVDc@|^o@;N;l(v-`Nh_wa;I5^pQbWXpyik#aCPyWUA0C#52J+WWWQAo?=~n2 zwdVFI(g4N!yS?zo-qn8XAnp18M%NN{p>Jt+#PE*IfJIcmDjD{dT2L7Iey5mf3k5MR zEkmc1MP5q6lj<(^b5CP@jThnHuU=XG&`xVpv+_E(m)yb>zx$%t$LhWnN6oaB=kRvO z$qrWbV2J#1}Ywvk96J$V_mAw_3~&=IQa4yZg){?30IxN;5xt1Yuem5c4n*_fP(2TOnj_sljVvEX=DXl!Tpm~ zrDO>93FB9%rGAd=tl?>-7T}LMK)yGmztX(^TO_TzG$2~XjXC9$<;(}BfnG=#=!V{U zWFh<+ArH7Uv!151vI^ZFadzOQ_NP&vhSncP8AcpMSOHDa#;AfNMav8?zw7731_f0Qpi7 zP)L4Tlzw%<%;m$VfHVX&*MFF8{s*8ixvA=^7We$jJJ_}G$8jMYh)erO_l=prM(1CY z@tKs3pAKei?n-hIyd3?v^60OrO2r{DycPW)>Uj)3bwG57aio()uZC^acFCJ^tLfD? zaZ)Y+LQpukx}?L7T0Zs!#H#RYQs@HFwrrhYM+_+IBz;fI*3@~%e5;{U%^89wo4!L& z;tE@X$t;Yg*QE+ku;|ak$o3k0RD8{{}ao|SCy0a%jxyCQErBraP~{Gf$3VOQpl^r9eZ$Dq?cH5`H4ff z(h8IYF1}KXQE69ZyjE}yi26bE5uyH%fjSj*tZ)|6Z0hw55IQwub1|A~pgt&w@v>b- z8QX2*bK}&9U*Cq4tq zCdPb^>7qxcX#(Uu#vTLkf8+GV(Ua#IHcsPGk5_>G9MeY*_AHeoS9KC9=!$VoJ+s$;&va9!AQpQ2N-SWZ5bS0zg924Z-`j} z+7-v%-;1HiWJ}0DU^ZnxuS)CE7yU)EZf>!HB#&Ux4$nM~Ecg{nq7LEP7Q_)}7F9`7 z5JDGvfpmx5_yTQ8wd=;)mJy$K0mpBzSK}I$Aq;t5&z}dq^AY8~wmkREMGNLpxi-^- zRpQerSBXQIMSE1dShn6!`T=r=p|hJmQIqkZ?o6@TQ|x7H9U}G%feRg&oLnCJl9E&zG`I6fx#Bky@r*H^ih3KuY-Br|NHuFN_QZ=zF)L~*G zR5^bTwq2JQA>mo|6p-P_IC4eH8Jg|Clb2b?Bv!T4%AdIDe>*XAg&yWSjE_>T+WMj9 z@-u!%Bf_yYb0d=LX5Qf4t>1outxM$gYnb}LFL3k!_ZKMhGC0+9i$Bx27_zf_w6H7w zRm@<`>cahb8PIAZsl?yjo=K!JX&Kyf@OTn zDbJ1489H*CcgMYSsZT^5IwG;PRfDOS*r$uNsr=l(Mha}){uJ6I4)dM3+Fhut|b^wI-NpFbK0LOXZV!A_X0y`q#bTH}{`cSALO;>-sg68#@ zt4q9$cCjBoa6D79{<%EB3^CoxL+O;g^}k3+y`~ctwPXR$v^fq810Q+aHy+5!%NLs5 zHH!&{=@MV2A)EAyEA$wItjw^hH8e3v4!tEC&$FZHWl6rIp>{$;@RgP3X3LeSm|VT7 zb?k$Mn>o~Db9AukHpi4>%h%QP4dViB`KL7X<&$Y534?q$uL!RxeJPK=mh>=Ob&bfD;GfuJE&dwc#m_p}D) z9=R7PosNYp``ua@cpHgKG9iC>j~;*XWc74@P1gD(x1d1y?$@`-OF+nw*Su>!-~qG= zy@qPa<-Vx_>bvw;(#RQ47~N${9V}9N^&U-B7vVMUnS23b!JAE<1c@=%f$*dEv%yZV5~k z)HtLFJe1KcjnxX~kzSphJXmcANNJt5*a2aZT|}$k8EZ;-_h;jv)1eIXUDWGzF3*~Y zA)nDp1fj41FR@oIDSms}xb{w4ih|HyPh(p(d&fGvB8=Cd@g@HdG6Z2K9iO91*jjf`tR!ouO*nOQ3)qzQ~2Y{t^! z_^5cbtgCw;i8nl98qmpW#~TA={s1D6c8tP4Lkj%|Ln?NYVy9AUry*QR-#RFJ|JvJm z^E?6+Jq6UA!6vLZBbsp(QCLZZ&PY=DfrL0lswoL;d^t60#kDF!Df%J(x?`)y<^~xM zP!EJGI)@h-)@Lzyp(5YisE|^EWb1QMpa&*By55`sik=5(8q@Qm7EwQB{4cPpakk~@ zJ!96+R2}#UIZE-fdc_7W0Yd$n-Q70sskjOha{_E2De@p(}zxsecfeNT!bnb(I?&4 zOoWOpSEXlNmC+v{d4u6utW9gAC5GapXGStX#$*_5!f_6y@fUx5soyONiw)05#3?*5hnMXt5RSOv%g;Yk5-7LxjHUO;+lX{)$X>*FrfRt zF4x4h#1Y$8RB^9pCo*$;wClEv`MdcOWd!mdr_vv6fz6>uKDVWS06*pW)N@g6Pd4Y}~vFz^~?8TcRpk^wVno!jqwC*G1-Ne+${^;Bw8Ee#ppK-UbY2z*}LL zeHuaJpESY%V_zXvJ+?Y-cte==cf&DI%h>bn)syxZ3m^%_{Kf0K$N@SJMc0vDovLj! zZ?UX|2XC8(gx@9K^E%wIf2qlDK9ih%sfj$?v+aADa9JjZCA=xT8#W=zJF4eNxV@W9 zbhS)ZO*UDbQXK1l6vU&R#*T(Bjfw(1h{x|F7Gf%7BjJ0X43F z+RC1)85-J6+y47Cp)c~PTyZG|Dh4(75iwmS4#ys4CwQ5#_@1HLVU@1B@4ram*fW;_ z%=ePs-mM0?*TiXb!|1&%e3sI6EXcXW&`L`?u%mzwj=_jQxvuDGl>1 z!?0-iaV%%{Z4!NUNpi&98oZ_=BCcNFc8F$dSEZx$H2G~m zRKvG68V-;7kEp7FEk9}kcxzmVx%9JoF|cb2KUK8x**4Fx_pHkfMJ(>fi#nJ9y@%<1 zit0-ZO>mLi8m$GV=14m0ul*UsZg;ijT{ceA zEuQO9TJ)Wm&+V0MR85{NQX?6mVgTfP@%Emm(KZdS-}q5tNtI|h5a|7Q8V0vCnC(Pk zJX-PV)~3CS2KGkT^6Y?A`r8*U;|{zrtO>4PmC6huj}v5m9QoksC(t=dz1ArkP-`rK zsf{vSrzS6JsKM%0ys5@1u~td&dRl|kIg#1vDom|F9+#?6=F=BYWv+!6=E=tHGpUK( zL}PGN!K^4U_sefX);sCjJ;wa}BA+w_ZE9Q;bDPL6HKchfco409?*{vx1mY>}segAh z-Pw0F-AXDjRSur*oN6dpfXcg~&z~_JJ5gU<<<@q?GG^op|3}gpZbRTrvNm7e?I<0s zEHG=KhZECN<2XY(#Z8yq=fIUl3(ZUQUp%sLnfZKZqwKn#U@#$~-JlXQ)129Q#l=Cx z$2LYPSt3Lt4OonxvWh-SQqM(r@w(gQG{>#_d&JGzHGF)u(v-=kv%QKFg+hUoQBii` z0bgL%&XZT(*$XmI+6ys%CJx@t2m`)q+i>;cvB<^F7%>vwFUo`soH z@J8A-Wd#PF+^Q1MCl8yPiC;qKRtD{Mk^nY#rRfXpKRip1!UZy$W**TdE`BG@T#=j{ z6|`z>XqPnU&JaolG@vS=15H|o%qRMN93*YhypOI%$h9YYW%Hlo_WX~gr6%jqoOeU^ z4itN%#_buSN9Zc^b-FIePg!+|gD!JkFmQ4Oe2T65rLHUBB<50g`ULz!yf5mQ(j_5; z-&`5XDQw~!3k-3;uQX@QUC1u~K6|QSO~5Pv#4YaNp<`+?pPZ%DQkAlt_y|vsGF2=` z_N`AoO8BxRzU4(8zpfT;#^JV3S;(-?>;+cx<0T~UcnMHxmIg}9&`+3eUmDy<33syE zQx(NIt>g(2M@H znnSPJ3+yL(;uH1>5|YF81~BBlUg`GYug~l)pr^-%c5*dt)u+n&oD82J8XA}qUOU30 zR*iQ*S*nitzXd^EWZLx0QNAW+JKd28YdldL-vAM3vsSRPeGaNnwW!G=voV6(toh+_ z^*U!O9|l#7$Ao$CvpBX|nref@*~Hk$5+Dbx>_k8t%Gx_Ni^DrDyNpPBq+dwhVa z7I*%3H65|I-H|Or>1P_K6#$&KvcM=gIhUD(Ez)z7;(Bye0>?&|%xGMwOSUg7^U1hn z{}ScvK5AOo{~*)_L4kW8#QjyaVmSGO9GXZ%r0`=cTfXdUcC;O-OZR_7xFIT5w^Z{i zCqFQeP^Au$G+Wn5vM|EFx4)E`x+8A4itm;sSo8Ex&On~E@TAB=TW(!ojO(x)=CYG% z4v4KNsafJP+zfYi3>6!7Jl8kxgbKVx+c|pV+mhXRPL`;Ts9v$#Je#uXJ4T`FE^?I> z1IkjpS+ze>J*GFK(1c5CR4awZ}5-mTrkoggEg%9Paf{SoL3%93o^VEZp7Vf)7c-r zxPSLeeVaYlH^n&juf8epT$l>OovO=SDIymH?Gx0qR33q+(|MqFgI1A9M@_+tx1Q>& zhazelbEN*H8PhkofK;c^p6^*=aHIx9x>Sp@so8t{toVky{l*+mX4cEDHXbu%6R7m5 z_3uORQBAz5f>cYR@_VzPQa zAB_t%gb=JiclOcv{d*G9jdNUVBPS2Rk#y{Lem@WW*#PpIK%Cp5XsgG3p7@LWM!TUQ zd@%9;O1IQ?yVBU_(hnXTTQfIP2>&V{=RhOK#thT~+{3z<;H1(8cJla;i|M}vpjGc! zPs3B!G&1~|?!LdQ2;OdM3hY}5U>1|**2-Zc>pgmMNpf!^uU?*1&V3x2-p53NF0|h5 z9nA`LTY7g}rPQ`tliZw;?Hk(XO-t0&iBLQVJfd$#X1BZ1}(%C!^+izVUU}h7PjgIU>z7^r+-8m!Me6 zet5?A`~YfiTb5_Y=pR%b#`ns&_wDZLoVrJ#PnZZCTneLA7VT&0Ardf++}NLDmMeER zZwe&;ixp~@tYafyKb8D2m@S^a7=phNv{6jGpm*N|cpUldx17^e;zMlWw2tbPJdJ>P z4s)3!9hDlYM}D;4(aidlsR98zNZN~2 zpSXSOfWJ$b-=63ny!MOsd;j{k639QiD(>&Q8!GGtZz>&74Ih5k`%H;sd&9#6q95-W zUfy2?!!m{o2N~Mf7049je&AlxL7yS_PXLt%dWq%gpYo?Y;Nm;GCwLmZN@mS1mOweJx=r~z-tV4VN9FQT#X z<=HZn&!#)z<{CNZVVr!Jas~12iNv1?pX&Ye`7+|O>ga{u=&OHdaBlz*TUVBjycJAU zi_FJXVDO{~Ptwla%M5;NsSK@)1d;gf)LF5|l^sbHYaQkGn%#wyK&Kr+B93yCWt}Gi zM1qOxH8axzS}mC%B8SiCWO8ExePlztq%S`*?CPe%Dea4XLsX&du%Zk0#&N8Kv)i81 zywZNn96%L*iyxy9MLYQxn;TNOg9s1rA?5$5Bxe+@;o9}8h_2qE7z-~!Wzrihu>{`W zvd9_p*f)VI+F#m+AV3K**~F6*EEmqLNAfi^CSaBn`;7@CNU#F=Kr@-*sx_;yZ<&RoZQ2guk0^!YcQfmO>>j0X0Ct zlhX~=(?8V1SdpM;8S;uoioB&4ZiBZ8D{xdVxUevSp%C4Noz1b17qLntOZ_1l? zMPFaJIK?cLhnD0_!^=ASy>l=D*3CLNK|8k65RZaN} z(sZz32olk8!4t+16iJ!@2!d6)Up@^ya-m>wG{v924~9(OCw@*&)$yHU$*)TL+3at* z^3v3Ob5+jK6Y+$2TsQ}g5zwdS2M3zvPR30KvJ zLlXrrrKTj?*&dj7&?@V}*0^aUBnUUkH>+Bs7E8~B6Xn=DVEl`{?N1sMUHpDl#E)F> zNH;EhGaCN(>#x8Q{Fji2o-@-_>FUr;_u0+we>|GvUAG#ZI!&HuYa9KZ1>IfFl}gzB zg4>|K{(2~SS_UIUM4bsxOK>gFFNLVL(JNh7n6dCs{A0dlWk<*_YO zxu|3pH%oeBq#^0+*IFqhKK@FiF_986Wtc@0qGr9NGUBnstZeT$S3k5W#gOqm8HGpQ z9l?9v?+;ZzflNHTfg9#9ShC~-UWkB(pos4y7X#!rR?qHl0=8A|M=gcQ-se>%erX03%dY zvuas44O%=ncpHn$foHp}7e|^9Ua#9@6-x)E(|k*>>kYGH#Awmvl;K@xN3tU;^=4n1 zn`6Nns*^yW?Kfb$L1E2B_{n$lY9M&^?>*2VVYZ_^A8RSzvo$nptuW(x! ziklj@04O74r!4#pzE+aK#l zYinip9kQmAzN~k!Pv)H-mwAQ(&hgU&(D<^kBG#&?FKnskEBLKN`tLXnJ+_xHViWCF zic(f^Zel6Mi^i{sqz<_BEMeYi%1O|*RVOd8Z>gAd;es3k9gh-qSgiwN zAMF@RX0OzuPvZ=qCR5q<(IAB1F7yfmN^1Hn;D;_mX(dRVxG3VB1+*ingPvZpw>J3m zeqp=8Dw2p(IKSpfDd_mCmyOx`K8z!=G}Yj!?9WCa|$2EiWKsA`?1BgUpZBY54D z;PHHxBZ1X{??%&``%pZ1T&mqL2n464Hb-sxGl{>@j=p1aEftBU*|wYy>Y?w}lPQ{2 zdbacebRQzYqn3@IQaSi@&vuInOIdKx?O!>{K4;f{D?tHmg!CRR#c~1MiNTLMe+9mo z56Ie03A7EMn@fN=)8>xWqcx9@n69d%oaWf|u=UjOY2U;1!}CVz4*io+E0T0X5f#rS zXAn?g6DpN?N0GO4_YgV>Xql2oHQtjy$0R$_8WlIZ;Tj*C&6-|p;Jyg=oK6=|k03;g z+XSXjzQh{p^C~N$CQPQwoYu|j>BTlf&hdWW6w4m=!~ia)<=(0ZC zZxH1n7z0=HbWeFnbX*4DI}aI(F%4I^gci>u^td3MZW%k+bN6mEKb&A_TilhzOc>tt z8)^IEW@mO$C0fYZ##1)UdHHNlq{Hq4d3S{%x4sy+WY_z(`dW3jh8s!NHG!1+;M2%~ zS9xj3)AdmCZv4u(Gl1-rp5N>ZR?*p#NFbPQBA7>O3221z>C66>j{E?WYt z_-!j^--=D5r)k4CkUX^LEwv*F75dz4X0nqnZx?a{>DB~rZIGL_9>SH{5@`Kc3&tNOWbVfVv^PMY zYD6kN0zI}NH5K=kKe+8~luWpj{cdYzdZ>AWWome+FMcxuYtN7WfyrTdk_mD>ocWKl>qUsTlnqD zN*_~EPtG;2CZw;`FO4JZt(()A-9DYs6Z@oreqOHV6&U;t`;56=;)a5Hj{bwXz_@dc z^moji@}7l)l#Aj~$xZAu%$YZ=?YDsi^K-~ zNzNceo_}bbPoE*Jf!ZaXeY%23sKd!C0A<-QV(WCiiWJMt3< z<}(T`vg}QTGMgjzpvyAF-ybUO?FqFtSSoGtXqreh9D=Gjok=&a=%zmzcEWC>d7A%X zA8ofV8YM=%z~`8>hHI3E(mAq`+8){sHjXgsdPGf3+vqJwaM|-oK|OmLg?p9?9I_2Q zb;G`>=F-EoFZNb1x<3LVe(LV5d0&b}sh1eqLHTQNgE^B1ln+xeWe6m&O+X6U0IU*8 zHfYEd738z~&$qS03r(;QE%7c9280i8XWod>8se`dmPgafk;bf;*>?ro@FC?LdHbny zo{Jw@6`_mBv91uO*oLJdsYOfo4TU)AvV4wxhB9%*|HJ`pddl|u`fP0YU{udbBP%;O7}>MBdq@kXg@(>%^-jHWVnBfZMlX>2}ULY zQ=yQe5HKW>yTRd@#k|VucXRUkhtjSAvHZNDo}YMyH?ho{c6P4sDg~-m&8IcKR?&>` zz;d?rU*h=G?zvpLqSNbKwB4KPyP=Ox0dJ+7AEM3=noyy^V6)hr?%=al zMY`-lSl61!T(?g8oL#DyGEHj&d{!%S)_jd1EJx!=5)QOf@2aCz*rE{`Ue`k7f<1S~ zQT)bJFLa8Wy@sa*dxtI!FO@07$+~*oQI|09q%;l^>O!VkbQ|Y+VByQO(v3&yclC~m zis%D<$1Ei;Me$DY)^=nzG}A!E*5(-!eYGtKJ?wzdcT*H|F{@M8shB8x^E`52GbB@D zL(=IOEY8CkY8QOD^KodT2mvf=&xo-V0|q28H)(ftoNTOJ?yR*xD;pHYY@_D(o?v?N zDvb&II75&isTCMTMEf?+^ye8>lKm3@w!Iu&t5)7-PL9NRm;wu3$r;_*b1l^@Hbf5B zN)rxouGWg5aWn4Dh%tH9yvNb&$?p?AD_IKzBHo7ML$r1Rvn3%$(tq!)qm@I(3Sp@t z7>CqFd&x>YDG#Mb(R3in2+#EB-W~T!CD-*uv*}FNAt;?s7bYQU>+IikX+3Oh;Z`&v zKOtb9!)8+_U@0rmVx&{ex|=@>V1G$0X1n4UgctXJxq&R3j)AtoBGPPLs}sLt)^?+z+U zy=t~VrwHv1!CO;G9t(;9O8-93>&laKH}s`|;nak1H1%>k=2_}W;%t4TTCi*j&j2V( zOXRW?pqaAqgACfB)920#{SLJGNdoVze4ypNaq^m1#eytgeXhr*31WATpY8MZtA&q7K18fc$UbU3v9q~E#rT&TKZt;P@h zC-h@M{w}S*x(p=ID5_)^Y=Y5 zHkBVm+%q+v-kt`jlsTANCa|jVe3mvL+XSNjnwae-z6SPUR2DAY*44nB?y?wL+KfU5 z@cBF)Sj&vM%E;{{rk(^J<%5q?37WA*_Ls4o^X0Jfcudqw4MD+TZc2~nRjJ>&YIy$` zjglbdL$S@ErsZh<18oy{-MhMJ$G&@KC;noNn+xm4_)!FS3UsSIehHiNyVIh3rDsL{P=iTBU6mGkN!cI)K4 zK9Mi7P7!yO_J)^O*Hw6q7#cJFih)lDd*8r~EZ5sv6x*r2zP{5~=z+cfD}eBJv%0A3 zmN+*_bth_>ljtAZ&o3MgvgSJ{VQm_}n;ua*WRDi@EK3DX)^dHkvVQxOR(&NKt7ifK zM^VCQ6mr(`6PkH(w8+9erN1l(C*l-&vQj4IuW%Q~vwew*Na8{29HZmfQ5H1fI(u8e zmPFiP4O3z+kIH<%G1K3upnY7(6+~W{0EK9sT#+y^NeY>a?Dmgk;IZ`q9+Ef8l9IMU zgb0-Wr$*(T%a7{t(sEUDoAU9QXPp#RDzP2pWfm!sgLMySW{zdh^;vhj+^oqrxn?n( zQ5pP3MSjGvK!p<{F8;l!4YK6!V!ZMc*o|#Ea%w`S^1o6=#y}-+F~*5#HbrJ$GAJ(z zg&_#SANfX=vg%g@V$z^jT!J8F#GmxTP9exP0l@-0$0}v$j>>C{Xcsl-&`Df8Tk#Ge zpAnXJi?*z}as4B)(#>H}Axh9KgEavEyd0fiJq8gPvqxy{OC?1c>43bR}V21+mVfb`C!7A6%#4$f%Zu@b*gwssyxN`@G$G~ z@LKaox2vLXajL{yhG`c<;(5a`-&e^S1EvQ7Tf@BzN1zU`&LOhK$4w+@F4zuK#4+7{ zR*maYI*dkIw=FK+-Dtg;Z|z&YF>rw-<6gT3O^|~l5)dta@dY~JzCB?1L^5(Ac{>4% z*cqd-zUBI2`t!ySl=ruJbc;Szh+9T%hkXd=84ZmpOuYyz*s0&WFPkkhAKhzOuaxcY z5^~*WXSz^;H#&{E0k^+TDsymmgBXs8yghMyIU-T#T_6gtKJ~1Pu z#MlN`wVmb<{izB=NQm#?67wiBV=}nCE^FmDOx6UAL~2P_o-47%Q+S}5uL_O2$M`cK zxp|=a!Nm73<_GjLhFo&}F9>&OKf*~G7kA<6PX6GT`2LxqBkwu#ukyJ*y>}5<5L2J@ zJk<42eTe1sB6>0WM%x+)72H+@4`tFnn|oXMqtbPqo`1`SIF)~~b2`e;RnS^Puqx@# zPlt(Nz2@@yPp0M5jH?2T#p3LSqzJQXVvB44UPmG&yCS93+4`Dw)F#$u)y!_1HtCxo z7Lg*3O;Z#o}d?3*mJ_>7+Fq=~-croXYcGpH=(amZ+F*L12ls-ON(T%a!$ z`UJ}uAF_qBrxE;sZ2My2PtX0@bk+BrqyQDy7Pkp!e7To4Yc!kG9VJW6y}6aeWcGUt5N6-y1yuYW!|_Sd(+WTlxKwjmK<@OIm3v zvae5he0|8!#+j?gbOjqx$qZLnzcAm+)e;6W^%{h%t3tfI@vpNP-K#g>Oseo8Tce8m zW&|)mt@jYVFkpE{=x(6Af~X{|Fx_o|9WOvzZuNstTT_TswU?0#P3b&8b|Qz}fh&^k$lSmJv!I4?G^3 zbWA)y#N3nhNCsR@a(A9;a{&^7{J;CQwXWSxK-;X37CnS^;taLmu`-dzE6WtCn<_+3 z;B56idLBSt{Nz2vL%Z;qOXjfK%`;A$d{v*4{4WZARyc)+vXf!>9A|+(~;en7p~<2!kQzZE|cqxh3s}C?!@kanq$F7C_$Pomk$+P zu+O`Zu=GyO7;_vtJp&&1w7*)xP-U=6qoV6d1FH}VmnUeDGLB+Y@G=xaaXWOwH$8vT~WOk#lB{q63uJJig$Pztd+4MNC>`9zH z4UL$#!GKt|&IW5A4_QA*%M}vMO)KqDnbcwJ1GxW8Q}5I5XrO{$hHGV)E$ zUF_46X?VLix~^8<=!{puhII0sOiRO$4>)XaQcvcB zJ60;YCVb-Qgmu!5TaeQcC}} z!njjgRs+}O^Sw)pYt!vj;X(75!d`b{tF*t>Ja};FN18@UQHrq&&>uTX7%tGVJUyR$ zn$d<`M2o+)6f`V=-=EEl_M5sQeOds^S#hVOqAX+1ai*^c)>b+fW3@Al@@$9_FqaxW zuD-(zDGtdVEZLgRh$#X{=N&C+jYiWs?OD}i4sZ*aE|jw-Z5A5rI)*wXsyBA=5q|aiJZ_e1aJEo2GL|z#NMBBRZbX|7NInnBDrH1*>|8$i~e@^~h%pE)a(4AID?x*b~ ztvM`NS*fg`KDOJ>+%`Aun1FAbtHAwSGoZ?7W3F3b@mv_W_TXKdIguFO)Bz>@)u#M_ zeIr*z)}8w#Brb6oor+w3qAWXlre?bkPoiEwmqza6Fqn+gBZ>%*3X;i!`}rSjK`krN zX-77m1#tPJxakd@^S6)oV~7I_x#t7+S|Q@+Q1>fC#ZH6S;mJ<9(+VPJf6vv^%%T>% ztoi=i$?MFJ60F*S-USj3bca-Bu|?ie($tfDdUuRZo7B0lQXQXjY8^%FbZ;4~`j0K~ zkNZK(TKI+qt%;vN%1SNM?O3Qd1S#B@72fUHsm=-)@7x*|5SU{sQ-Y>qNpaY=F?v*_Gu9E4o1l&!|mK zLXOBFkO4O%h2$f0wIkw}>`i$d)(09SO>iW)B%NRXgkOhdhklmCZ);t_e!JRuvP*6= zsMp$pQ!_&iYCnz>dzCS^{_dEZK0{0QbN!;75 zi~v`Y_1fN)n3lmJ8*YdbVRor0V0jm8m)G5!pOEvw!x7z4;wU5hV&>y*i#3c!Je}d+ z+s1ifWsaqRMe{-M))AR%x8%x?BZ zH~j}s!UFxc{P(skl>Nu1#K|CQ<@Tb@lheu`MxH-Es$X++5pJ|~Plr`@(UZDKgKp5l zZs(P@%cOf_yUYEvGUtjk&Q@MMw@q?<1mrYoa&Z+to(32_T2A=cOWs5^8PE_ zZ{z#(hmUDEe++x9e^~LZv$7jAliA&r@r-m=Du}==Com*`t!X8h9!k_QKDz!RL zxC2po+ScWkNJeFxtrX6XXp8S7C2kX*FZi(GDzc>fr~=qGmOOn>fTgQ_xwmw8$xHX;pjSuPsrDxqE-cz>^#BpXp}`|r z3PqGoa^y?kptlDv>44ipi_1$)XXI4uK|+!~PBMJ*NsT-U!RdoDXeVb_T_DeY$>G!8o#$;pkn-tFoQKsNle z@QtW)Tu#enganVZpS~_O|3gKY51#lX(cp<|PkmyXAx_#y23@8NGXj& znUUO--Toi1x;5VRD%|i^|7tIbV&WCv%p*T7!o#Z8pZ(iU^_c(h2qpw)t4U(6dejwOe;;z9cF>X3(sUf33>& z>Xi)Tx7$x&%A88Rd;CPu>5rB@M|`7CJil<7h52P?)!S%73>yPU^88yIDz{t_5ZQ=)?rbw>$ z4c#Hq-6CBwLw8GecMoyi@%wb|wfExNYyE!foO500TK{-m<3Gc^@x0G-Klgn<1Ampv z=-z#yrxk-~BRh6&n6q2ar!{xo@K(4-uSIJ7?aOy3I?r=niyD6_LXZ2XHsOL{Bv(zm z{v%S4cjBJooF*RPt?zkpF2iwajCyc9i5ovs+=hysW5tS82jCCl!9RL4G`npt@U(K? zfpIWFhcJ_MtVxaU)jyQ5dvi{m5cD3)zWX1(c8`T@9qBkjZOFeH@BPm83xLpCMW1^eN)4jYi5vh z8TX|GfA9>>pv`PyQfX%@ zaD|X7*^XS)SBa|BrZi}a_7W?$SgQM3lUj5K9aIlegSCzrloS z<=+ufhI$xGB~jUJ2WEBTP2T4u$O)P3G(kK9TRbtBHoo(;cH(=aGYF zkKnt3jWJ`EN>|)Vs%w-=bRnC24$7>=%@C5l?RFcw#(rScO|aR&c3L#lqAgF;FFd-1 zZRNT6S}>_+&xFSd?rF@LHVS- z=bGh6Ma*%+T$@UZ1b?9g!VUQE4{U2UJlVuo zo80W)Tr_DBsE4%#l(ADlG;4^vn+|+Ll6;a67<7w`NSy{yfsaVEF$@@WuXUIiNr|F+)v_+qY z`;iY4Jb{!4RY}nEx!A#@kLD!S@1B`QxKmTUmMkfo9!1=uW?u1;S<+_f=4cOEgYsE# z+BJl97R)u!^v0=Ee`u@Ha5=DXciL%t0Op~rv@Ey@^=cTLd`X&~&U&82_EBW z3mZ?&-5XC{KKU+NK={C~=7ycmdEGG2>iA|iSdSnChB-8&vp&y0PG}8B61v-15R#T& z;qlO%V)Mw=B)tk}srA+AxAVFCD$JucUit-Wm>B}Y8HUvbRL7Rw*TyI-Z43$wY?YuC zrok2)HlYq*<0SErexWi`EXc1arCi?sDup5AD@kQ6y>^vAkMO*+FI`jpOnO867)vlE zL2)F@+%GSH;INkjnnn--o#APFbIS``$sH?j+<@qk&Bz(g-4m8I*>v*P7bJYG`)?&o z9|iV*Dve0gVmN;iWZ&Mfnl%;)EM@+F90YW2g7b`6%iwlUCuQ4psHP^kVa?6QkIXVG z3A=FD47Wc|ec!WkD0%5M_%o@rnVmGE&v_9F`t&t&S$Qe>wO2H=SprnF(fx)4X)>=pe?plB(I`iF})0YO5b%)!BhR z3iR6V(#ex<&}C$-kj_nWFU9aVcg00lS~c_JAMJ?g+kai=OR|}}6PU9uI(ss4EI?hH zr>?L|%GY{apFkdX!PS(#_yGGXu7d;wLyabP>|mhTa)ApT{y?$T-UbG4$OA;=9oEJn z*DA9XF8mNr_0|9kL=d@i&O68ACXXEbrbE^g606vQnGLPDsFfZ{3eVkR7JF=s`M5X@ z{>raM6|k7W_9?-t`(@#R`yT#`6{y3#T6qA_(0}m0E#JJXEAgV;$IlV@&1e>^^R5q4 zwadnUiL|titFK~`t?&B?(RKLhYhUXeJ0Gr%dAMiYJcqA1+|K9n&OH1R@;^TKeX5(^ z^ieUl^C7S|_b~xk)4oigk=969b>)Glt=`I9P^^mTE;En)>CYg1tFt!X zdc)nRS=q`*1X8`T-8AMkc$+6Z`d{$cdNYi-j1(rjp6iZf`)=+I5bH;@_ z4i-#_(_1NVr74)dx7VOaWVJ6UJ+eB=9q!i+)X>_D8eghtl;?B$1!i zytUJefowOBxlGaABy(Qt%a`S2s|qF}mFCQ5?XzCA2b(tL{sjqINp}g&o#n^PR+VGvnH0PjV z7hxI#H98s_;-;^mV%PLE1ZrWwKMeEN>I`>?Tfcyc-7y&EuY>*mpuykWtCP0LBRxY* z_V{?X)S_r(jzq0U_l+V^{Y#(JOVN@Br4B9By14cJ=dhL|>g3|8PZo|0rnUEMLo!xh zF*8hP(8QAqY^!)+?miKmWm6WdN}7E@uQmbU7_9nz@R!fen^F-Rgt_aP*&=g)z+N!B9DD`%AptL=$x~~QmUR_6hrR6 zyefc@KMTsC*uc^at#;X}b55Gf^1TqtZG*V)fz`3+gP($o`5wb+m_>JsGa`@hJUs97@+u+kn=tcC2P7<0qD%l6 zGBQJwF}gv>*tP2WPGoNYk#7J+1b%gdO>_!=&08jd=OwPRsq*tHewy(uZEtv88^=sU zQbfJ0>Ju=5sln9Inc$rw-Or5$q6%N`IHpaJ!;_d4VgDmh7BI2Vh9o1ZKbX$J#mHvz zxx41d?R)C#*=0Ee!S}4Zr<2eAcQoD-r{;BcUOK!!J;g_MWKFz#+@=vfA5nY})pmpjU%d`#(MmJ%OdM*`H}b&?rJtfN3*UKG$wC+YDaS zYiB)@;!?a8$1{>B#(YnH{S+pA*uW>c zu2OynpY3QvazPA`3AA@5b;beXt;_pqLFaXL3O5Fx@K?z62Th zy@qgi@jaq4K@8jxTGFVK4MLJOGx%HaVk~5`D-_{T5S1`LpW8kiR~Ktph&KRZ6B`J6 zJ>ljD(d3maeIwz9x!Y3nLF$DHGsDP`qM5_~pc?|5jioh5_3Z4;;E2@9{g8BYW};ug zXlBmBBQ(I6j2#-zwE?^n7r)oxo(1msmt%sXQB{OENAEgyD}S#&&L>4i zi1=LE!!h%jXpIA(kU>-gAsD|NCltD3ZH7}7StZooeb+BB2Cmwbl znd%5dTq*Rf8}AVP#86kHb9QBA zW+Jl9@R(XvA04__iKWGPwl1qL&f8becd{AmTY4{*DFHDfmgd4*by&5FRgsXwMkuZ2 zQG=HKR{itVDL%{DQ1zHG4iz$f`^k?j#7-4zF4|L{EGlHR)@NN+cql&?2pc`)7Drf*-f#S#>`5*99> zs(aB!INb*W zfGw+*@!>&65=qJ`HwXp#%zP97yp>%1@1ZkynVUZ_6+b8RW?M$wmUJc)6tfg6^kq2@ z0__yQKtv5gAX1zxC(ZRNZ6Sh zg^~{xsL93fGKjKd-lL}Y%e`Iy8&*Kjc?0O>yHf>;>nq(eQ_v0dl zRKJ>Oka6MU5mFh~=ywKWLkg}$)v-~6XJ_`aGuCp|vF)c!^$tOi8Fm$=Ach^W@R>^R zL|E06G0%8hESfRt;Ar_5C9MrYlgU*dPjTCCh*A|OBGPUfS*Nodfk+UC zBHlS{BfrCbe|UrJ8T^^ZNKUnLQyB?Hep*_HIDiAMji*U)hmCN1<87A3U zta)6!Ov2-KHBP%vl}J!__jD(NkVJ<;Ea#g)+i0UJZgaq~t60u6hqq_x@$e!aoIE1% zR)0S=KT*XlC?C=gn^JudaWqAa4r;ENN?OxzuUbxzv+y~*PN{-E<;iTx2&&Fbb2f`K zVnFDt+7)8S)GhMmw^kz8NK_5k~1ubkg3xmNE6@ z7ph9kU6b$|={0#P@*7p{!-4-sdQ*vTxdLxKyl3ZgP79B-2pk_DcQgfN`L9H*`8RTZ zc1M!eF8zDE|GnMkBL6SnZiKc}!YYTZzVXyYj2hVE{X#<@5oXTEq?iyYu~gB^c0G?o zzCRpid->jC?VP8D%8ih7>x6fFuzq|(_dSC^6YXCSVi4L_v6eS%KZ{^$c=M}%gkFM* zwLS5Ktf#*F&;YB3P2)_Bh_DNkZ=^%JvR&9pCYqr6`YTTl4v`7$E^V=Pj24_O?2d0R znHq`J0A4FLN$sxbTX!Zrf>ynMzU2-aTnBNHKT+ead5Glm03@m#TzR@a&#kkELdO3s z2sD)`2Er-hqehEp6E-&J)J^!OhZJmWZNeiM)Y=2jE8p;~b8pz=*!e`q;|h*H zy^I}zbVD`}A<8aml9g8QA3j4gEgA3ZJjtO$y$Uk-=^d!p?&?hlBc!#=L^osFYz~N(iouA*QesNxyCG7^jAl9-@ z=j-Z(pKaPW;(w+Q#y@)0b33F@XX2@eVCdVrcp8}@mD+?|FdFTnxOBKi)?aXBA3F1HK9$x7G$z*~={~z6< z-o^R&z=62s0@=!g4a4%H0f@FZ{Hy%iOW8NkSrL@?pV_Oko0e=R3KB2iRTs!8$JKjk z9!;~2vsrrZZ@Y<;F+GM{fKV9_u0h_sA!NuKM$!*uB#hZ9L4MtJdD?8$xB2-_0utdV z`D*aYzYhM#nz%dP=yHoqyCw4t+F8t5)$XAuxGp5yZD3D#5?kwzoj7l{&bkts!YQ9Tv@QTnAM`S|3JEQ_uZd|d0VFhOppxXR`-Ets*qJdrKTo%t{xy4^V5M zT1348{`)^CCDzQh*+cV-{w7+iX8?;)qPN*#c>U^dUlQ;emX}(lfG2u5S zy1%gD7IoaP04)~6czm+)r`XqSUUo-*yoso|K#I_OsEAPwQ@l_6itZDwJ^7mTT(7k_;}~UJ zm$BtbZ$>0KIW zg2u$gsiT=15{3;ygEp(_!p^m{?i3o=6O7p?i~j8DPp`QU?iWzL!k6Ect*;T#YxFox zT`t~H$wlu3wmZq?o=n$m;5WY;A-bA5AI_vvs~DHvZvt<2@C;h z>K05tPuz)4k$iTnJMf6=jfT6V4v!V_btc)cA4s|kpwsy;y3#LPd+hj)z47HQzL1a> zpkl=#qtMPefqpKRiD;&_9=O*SvZUd9u+W{NyVxVS*%$cXgndj*yQrhpw^&L_$}300 zW^I%v>j!s0H$p>*^(?5kS^#so?fRv-h~AQhB=Z~`GveGTy(QX(fwZEk7O?zD(qRE? z3ABxHlT*g_-GxQrI6HH|BZ5m~IO@UfghECRn<|eTXEJhx!aO0Fv3K1ol$vc65*$o_ z={e3b{o+AFyNhCu4wu^hkiXm-C?a5k<> z81|%!St?rXnnIabMZNJku9JgdO{F{Y_ln1qMHhRsSotiYkHtJ9DO^jNpMkM>4_$e< zd?gR{WuG4|rG6!ME?3Zsx<90iT)f27ni<$t27&DC!f5zO)#DRg$)JmC4N~vz3LiCA z@KfiOA^S`Ot(YBJRp;$C2qV4wr8efdVDVG+FhdqChWhxhS0CVa;1h$dAAd)0PKvJ7 zi>i}S65Z-8Rw*#Kg~`RFGV|0Zi!Dw|ZMAaAr6ALy8+n9XoBBaFS=Rp9)jJF?v%5}C zP-{(Ow!<{W6T|_WM!m^-%R@!Y@U2Qma{sm%hXx>iYOObsi{Ym!f;h`u#b;v5s5$|f zu7$On=1XURj-PqXmuV;SQLs5x6;a>cZ%Z+T--DfL|mMO3)O8Sx9E3Gm4x3`Pa7riLoM;r!KzRj@e;@ z^<>3k^n#AtJSy+UzFAZ(j}-T7+z~4xVY5tiI%@Q+jAuHYPLBAM0OjM{@KmGRe0&k= z@aS3S!$~&j$Jxph?juU&v5V0rozF^ZFL@(#4WZWO;IUnIxK*JZqEitUczBkaV72}9>Iq8YVz z;tVOTE%F5XshDZD?mw2kBqS`;mxX%hAozpkLYFA^C>LFUNuL8n9hdi@GK8kS&! zOV`8pLaSiG?Yr0X7U+0dnSMUcMJmC1ZoDmc0dx%Gowp+Maq!@3l$J#pSj*XyJBo7} z-oN)Z?SkEQaCUCZBpVZPJ853B5YQD0MpLc&MVT0yXZ> zrfZG*8rrVx4Ac*v;U5vc&YD2>7redjL)K+~!%>*K=tfI9&luaDNpG6*i6W1^SS3`> z=IFq)(C9~xwu)(?dA}MYQ(ZHZNs~H=7LLt}II}_UneM#6*1Pz&>NXACZ@brp6-QLZ z9kRCPcM8>Lh`0Qm9j^O7H!INuvcZl9&uk->rue`qrNDBB-fSiL#U2-ptGDkNuBHHF z_`2}n1K>QZLtP%n-Q1(i5aF`^ISdTQWb`ecS=K+8Wv*(B6?7$%&np+bgu6&JmZ=t# z?ZRzu(rL+Ox9-MitkAE0V(81?l_<%`fvaS(dUzB7HIQ!1T@bV+ z7jru;kM=Fn#y%%U^E>mu!azl}v64229Er2;JnRV8veRsO;?%CTbuU+A6_E9^^Y zB%97LP{}%L7704eQa&BixY8Szlg;WbXg3qq6NU&6TAR`)VP2~&{;r~l&#GZ&)}Oxh z`Jli$V;8}GRHQqXEv&ab?vii)1K(AXjszf}k7TR3?RwfjjeLA<7#1G8vyqxYQ1*19 zlO=Dy4;131(28M_{x4xG$E-Hc6ZnU);`Mnw$5U@I_$h3nmzmU|SL4O79nN%(76qDx z>dyT5Wk|JF(t@?+%!nR!y{b?B0^>>xks4}4|KQD#YqZVH{1HYf1aRI7*%6jhQi|kQ zVs4B5Hu zZ8l$`*UkLwqVvhSlb=R?v^>!9noqBx}ccYSAtL~zQQ#5+lU5cEs9o78Uny5d9yD5u9f0U z?bpH?b5Vh<9E$g^g1*RD=0Veb=p$`cU??Lll_aIo_@~fpl~ucNTo5QrZfeH?`n*36 zZF{IjNgLM^dUT`9(MHwg;x`o^p6am^8)cqvM*1PaqdV)V#pH~g6CGD+j+e%tx{51P zeowd~wc2~veMIo9)>4GYe__92ld(cXtTq(raVvBgoigok+C8W!(P=PWJ*B%QO^&wp znqG~F!072l-Po7V?=EA`*QMtCMp_mCaCX3;K?e+Fd5U2T-Pb7G zmAKIK>rl$2{?mJcj>1#Iq`vkNfRH0ZrB*JQAtr`#D136#dv(EBZ)ddKCcqjou&7ik zA?0e_vNU^j!8fWe#S;zV(%c@8IsKHtWZ(m50 z?eHD{0U^65=|8oVU}$axo0Zp~obO)kxT3xJfq>Y-tAdu5D)fn9a<5S94n^SY%QsBa zdFs~G@|{?@CJrSiWDXqU10g!>)8QT3&j|<~oZu@ox;^uXY6lZnU7d?MvrNp>Eaxf^ z#pgYU$1d%61fzB*@659AUnA+qO3nD5!BhQPAKSI|II=qXPyxneG(7xq1wyaB4dtj2 z1l6Bd<(JB$b)CbBM`m-4Q7W6$jiwMrg>R_|-}n5a7`Rg7#9NI}h02GHec2XJ8ZH{;ifn;!fAW!0jxp0xqW5%4z$V}yzu0?}i z#$WzGsSsk<;&3TnRlu&sA?}i|JUNsufG-dBY8+pkg48@r8_TuvaJdW>R?Mzi6|;I8 zKEpT~aK4=g)qiR(?ELN(!fVT%)kXM~_QF#BwX?`$-{agy4jwYz^o;VxG0VVDjun_= zwz|8#ospB_@JA$RkL8e^E$6Efb}GrAnVFf$kxHeLJe(DI=6O8;{pwX>q9O{C8;0!* zj+|4k62Zpr#6^+h#Y7j|KFKD5C$X|5gHj0i zh+ivYgYdnWt527=FNj48{9HTDpns|C| za4o`k3YcS+2ya{ zY{<~Y37q_dHck!&pV~E+bFJM^oAm{`l=|HqAF0-kejg)a(a)3$1-_aws%92)z_oM_%%+PjvzreDSx2r+Z~ zjEU?rLMOU<4>XHy{)o_m!;85r6bW= zPxjESW0Py#o*Vm}Y8(X8u~7SIpr@V9>q?S}3Yv+ps4>l@9TLAUte8> ziSAg;zT~dle0veQsbv9s22@muyRhp~^^S8>^uFHhS+5eLY| zRlq?=ZyuWpjNR^&f@{g6kT^x1X(B2mrzf7eeam2|;4sm7rP&sT=5WVW1E$$dZRMPk zS}3>f`sN#N8)T_#pW#V0t>oV$S&b-N)(Cf z&1TYrsrH#T=g?y6)l8@260cJJZwBpV1*4mHn%R4p4P=~bmj^_pv=r2PnpV)5*q^e@ zh~{wvL;vj0K((AGUR+5;+wXaej2>WoG$I~Cbim-Lq^kbQ#WmtFsRr7jMd&;$34jJ% zyGj`pKkaEdl}Krka(Dc10a{dPaxcS4;K$2|N%kN*;7TDsmTOV*X04O{6=7wskK&h7 zDJfx)@2c?tKrHRa#8oRH=Xvi7M=LG|$gG#~+`7qALQcbBq>6SR6d(A)2vNEzda1DMU5 z;bZEBwS=^?4#Gc_`m#jMhwulN5bJ@434os!070az4 zlhnGEXKfBKcMq(>BIlG)d7ekk>An+buxTJ&lq}n2KFVlS$+S!2OCdJS^LIl$9R?yp z$R4Im#@zGr{fo)|VAiny;NvWW%LYvnu#}Tk>$6UHL?lw_ptqalk%^c|R#SdoZb1qk zT@O|Z=56uuv-O+k6v$@xz73D)v^XUMSCM14=?b`tY$5j1b}hdIB<%28@QAIA+NY!r zH41%flWZnpjT&`m>?;mnU-HgIg^KsI8z1LdFX>My%c!5j<&qbAIep(XLft|l3;O18 zn;wSi9JF!e$4mXd(o*}b&reMgjtZ;H(gO8bNciaK>GN&Z#`0ZHh3KEnDp*-W&_APV z`cl7jWIgzw&AI=EYO+oD!rJ9^tx5NEEgr$ANUP4`vN3|+Bex?Rz9ON5Bf^PRjzT=L0`#YHqnJXZ3BZAh?3j8(VHFOtQ7k**>4 zuikYRtONws7?Y=TByF&o1S=cim;#Jg03yr` z6WuRAfdSRFQEZr$;a-Je3virhP72e~XPJy|DmEXZHJ&gp@+xI@uXtLgkBBy+ieTHi z7Qy5CHK*isZu}wZK~SHP4!~Ovwwnn&uWk2RIQ}7jGJF1dK$xeM^p^~gf|L|UBmWbL zhuKROwBRUZ;I{Qe^A_%>c*aqKTJcUuk;_>8W8_8sRP9tSDZRgb{xAN z3&-`e6kWuj9$E&qtNJ{T7XqNMw_WAL5H14UyHHYfw`q3d|5GQ} z?G^^8SLF^rT9f?Y@HFIAzX9t1o58ca>!~Fa*b@^S7}VDlYZ5L~t6jMX$EPS!{eWph zFyWACk}V3y!oI}eLgxyj(Hp`4dolD8E69v^44BOUiuSbwZ-5$L6N!-1q#yh=ovzW+D8O$L?RmXWR$@G=Haq zB~JeSFTbBP<4@u-uXy1X;cbHpU3i|Q;UAfBp<)Qu&gW?L>A#GWl7nUsHEXAImf-(h zvN5UFS(*ROtR&{j@@Pe~4ew9Cf?{7T z?rYzS@PFwM%Ep9o|BGtDfE%qbb z=lj&qHgu35k{FJ_{^VWRU#j&fZm3|;$;TE7R{pd?G==%fUI5&7kAgpl?5lflkv$^h zOH3%@7u`L=7u3-DZvvMH`(M7?M^Hg-y?gL7`eJ{=p5SB$3b&VN9FV+9t?|*tGm`dy z3ORtiMV;%|sgB8J)y;%qN6b~9AjVGW@mAz-CHbOhfF1S}BjVi1Yi3%v$5Xf0KO(M$11 ze*sGC0SM4Q()x>(P8%YoI&yqp`R&B!%TJ|VM6Q6j_W(mGglwYY^Usn|_@(z03@ z>4dn2Ayn*U^1Ez_7q;h^A4Y*+>7fdpWMjKRjZk|n=rBfwTe0>e<~-z9UBYvNhZtqg zbr%TE>I{JF_YK_xjIvdaFQnu=U}gcQ+{gr*+;{>n6K=v44nNHaHox@*UQS5DmF|Zs z)VTs~p`T^H;olNrIy0dGZBgd@ZS1okL_m8ly+z@!X)E&759n{6ULsCeT_>ble@cq) z`Sd*zGJBm+eE%saG5qQKh0lz517r`RJ7~cfzKeh683q{$zS`l3zscVJ_9Yzw!msSI z_+sKeVcJ~oz$?T8SL9>;nO6jG21G+${=)R+fi1Wl zOJ?_)HY~cCnjTE`kU^#}g}mw}?|a>!8n^?Qx+3f)GR5X~pX`k&M%e{JtfBXSV)=dY zKT|9%=AAtkan{2LAtU_QAh8Hyem2W))K?e+EU5D`k6w#5;3_C6Ot&@JHutJ%GBN@z zOZd2uMYm$S7TqOI@zb2?*LDpM^`2{N3_8VB`HOJkW{P2NITzpwNoV*M#*Owu)v%ufa`_!3BPY!(eIolMt$;k zaZ_6OJ)VV@()$8DU_*Mlk2?1UM>N32BG^o0qaDr6_$(&h_n7Y+df=uDD@aFj8{Gxz z@*ep!>$sI}QEv0WX;$naw(So$6)H7Ex1KD0mDK~C^7!dd%M;$9Q! znztXOqqwviI|wfGE)O2-GsnJS?1g00gFbc8UMQ=z-h=-QKQxO(S#q8=#gEHu`-lqI zS>A^{cVw@-x}2aLK|!Fcp@7d6PE{}#y}EW(wDD}_Xv?cvpm{WqCs}4a48@IBigc&6 z1%U=%2n^{bWj19|2{i5VXEP~YKcWCb>NaPWi(3y`Rt5PR&7)k{$^e%phmD-C-^oii z&SeA*y?t2G5)5G%_gh9d%?k&HhSG*M8WqdUsDdqbtdvbI!(Ldn!DQegB{7k$} zQ*fo?$tbFr6l(Xt={Vz}h@1A+*YYMJWY6(1iM`HvNH__Yk6yqg-cmmKEOe~#~;nX6Ub5AkRpQKQ~FB6W6m!)g3w<5 zF+Y@4JnZr#CjW4WbvjU5DuPE(;>BL)#F-vY4$&XnWHat4 za@?|cj~vS!2;kyGnFS@SF@*xomlcb=ToM;*BL<-8 zBp(uCh8KQ{G~L(j%u*FopESy-tArGT+Lt795-zB$&24(91W0Fwqr8A(D7-g3%noP`gL$yBH}gFVB1muiayoH8(olr#(j`(x zM|*ps-ATqT%7n&Unb|(1Y@l2^_vmb25xFbSqbJrzgI{8pv^V5DFxQ@&mxz?wE>Q^| z?N>mBKidwh0zF*%+sL}%{289X+zq-2Zs&Z(k}b%>a9UX&JBn!Vt7%3FAOqyjv|{ck zWe9}^Bxr5!34B)_PxzA8;VDIUx>FpN^g6!QvgRT|r0Fw-e00yh3G^}+a!zwxN2t-X z6V|Gu;>1emcc!Y9zNW6Mr2_pZUlN{%N@vFz4-zwOm6s~maHqpT@snL;9*GylQVb6l z!K23Zc=Eq3IjEC_G_sp~>ic zXx)B)bePY8y!UyES&AL8A7b@_bw*)6$Oc=H`6L?v^p zms@nVKu-_HP!mv~bdSE#=R5An8U4t%ug=>RsKnd_y>rQ^lt?11`P2RD8F#vRn--l> zJN`R}@>GYA`+;5J73?({caOo5Z9V9TGAmbd*jYCJYC8JGofPyJGM|?z3mWyTFAo6B zqB2D}nU#F~$lJ0vq`%?q2lJ+7zV4%`YlQ1xGF9_r@L3{FLI)4LF>M42@je3}0q?;V zi>r43k-4H%%kO(2=lP?W<52nsP(y3k7b=ZQ*5puB!=NE=c-4{H6!mVHCp?5bi8Y94 zuFW3|5{{ZkCE=)H%oTopD$?~;mhRvLWv(|qUP)9N%dMBaK{$c|%vBfBujbabQoaPU zE;PwxVGu=B;Dfn~Gk@<)5il&|&Y zGM8-WH=%pco=)D@YGGMiI}#8 zb8CZMW0=lGo1%5`nzt8c@*Lpd-rlLDzuvntn`3$r-#b5wO@DmxqH+kRhTIfdSzifn!1??BnPU|nKdKH}uh@>MXV+W&?v~wiG!5)A8+V1j zN~W}#V)!MEZP4Ua@tY+&?RTP1ME(qFW5e>&0#mFiBxsHAv2ADtjk!z&1(in?Lz1*M zLxqDdv+!v?+vnAWFr;Lff*uCmK)WDMaZ7B81bP743V-2xKkJjvZfq(1nwB$7dZA=s zE(@9!h#I)_Xh>q%IYJ{CnMFW_nZE`!j(>!c(8pYYFCMEWn>>o0Re zPabrRI7Qp|s4-%T^kBX2&L+}t<)lfLIIhBo-5Ig837+mvFw7%#~ zuLTsnROEbm;@*YJyWS3uvmFl1hR|~rj!;|jt?WB0K?r$bS1=5)WD80DL$4mye1sf0 z^B_~a2zd}9uolo3Q`#J9uF{Mx5_jFurW^3Bb7GHYmdq3aFqhE;vSUNbo;XuImrsTbWf<*-7`yU z_m(TSg-9@Db*4t|xLq_fkQpS+~P24BZsp zd^MI$G?c_@c-qr#xmKYx(^|rT9^SBUz0t>Liw2lrpJ&Xy?sOtrve{&kXi~hJ&nIJ9 zzWBJc{wvVM0G4(oPDasXoWMq~mJ!vG-$g(gfs^qs2(!aVc#bibuM%d2W!!laUC|aT z8T4U|yfCIbTkIL*K~J_4>vT1s)p9i(v`5gQIeud1XNITnZUn{?MZpB>liAApg4FZ(1$f=7y?PGHCVQQR*H&fLKaw~1#wG|(uY^^{1_n}M&#?v-7IH51SEZoI z?84arR0!!4rvu#v?Dhz(`C?+Lk z!FW6GTaZir_u&U8#N2NZgIS!@(B|ZT>E6wh4LY#p_xES~B@ZiYB95uu1v&V!?0lm1 zKi$3aA28U@8h<|)U_n1|Qm5qKNZnt+!);j0`~}+g*T|;yo9WZW);rzq_sWJWJe!kR z9k!qT{%I<|RW$}seAp3EOC`D^ho`VLq3XosA#OimY^JOCFz_=!s|qpNI5WRkBl90s zjC8xga1Z~9m)qxi6j#4cXa-Yqzse!IgUWbGQ1ghDvSKCdR7iWlx7|pjo-=p1#t z5afEilhTuEUoL2YJlt7IQuB5ItUGSU(+lK;l$ZN?drHkqpSm0^IHsvu@T6&F9abS( zKH}$QIQGG>li%7i3O(fxOqR_qpd)*_r&V>hXf-%kdfjpQrnod11%@t713WM7!8xsu zV=!#`6(Kk1t%*HD#d>ZfyYRn5YuM^wi6J0}+$8l&Zff)UrEe!ybx zTlmPt?8+oMZlfv$QKH^`#W_rY*M*NBToM}K84{bjcyXCu{jOoNNWqQwhA|j?mxRUH zaa=qWaT)?h>+X)PsXeJ)&8dylK#187NTdk<|1?@21yXT((&o&z4(X4PN8n!ie<(r*{C0ty7M^4Jpn?X+9Uu>+~ zieJSrXg2-%g2}*-+2dW4E^ae!@HdOmKQ^^9#P_@3$hOZHsI$(YXz;*qnh^KAYx`Ra z8`oo5etzL+vQl*@)rQWp7)im#&P2{~_y^L89}!(aII@zI6&C(52N#_++4GJAw`Pz2 z-d>{r&+20I#pQMxhtWgo=4)M(rI+}KI4>^q>2S|cKsh<+yx)jD@>`gX=nTVYZMZw~ z)E`#*R)3pL+|}x{+laPnkY8KTf1JNXNx$C*GdNb)PzrCs_Z?~X7Ph1L)q%R$`*WUu z?_Prc3uhc>deOck0am~0TRU_6_+1HdTfmK(G+6JL4u2>^ZY%01H2K^U z9PNMMcXyNzUb93j6zO%m+iR8=KK0*nSE!D)vUI%;E~4u(hxHrFmf%EJ)LoI&We$5? z%a+7sR}`iA>2hxX{foW+u|&{a`+c5*90C)5AG9PoqR7C<4pS7!?W5Y!D2&})x|V~x ze>XaTRiM80*-bD)MxuYKgNd{z5VmebT{mu!+U7>u$VQ5B5(|#{$zjwRpz`?lxBYu1 z&pIU^7+#!ig|W_L8C_52x{-}6-1nK6@eup&ze~};X8MTA=7Hbu4`a3uJSxTg1J3)y z#0W*oZe1M^Az`NTIOCR5Jb8QW;xf&-52hT~TWbUE{r6Zq%(W;UalCou>cFvY&Buf- zj^2-iT@XyZWLd-SxDB1ZKFp(V8#?x`u(O}DY`Fq6w0(5m(J*1=`&l6hQ|iWNw+hV9 zxu&!6YP@(f$niM|kD7MS?BNT9@nRkcVuSD7;aQkJlHsDiwg$T^zGg$aLoFWEwJwBj zxou`3CC|=D^JyFL&~M@3LxpWNcoQx5AD!{Qfxu4fP9u$Pavj;~yz{n6DIGs)$f{_u zx4yaN*-DTp4fc9Rrfvd7AqA2mG1$RJH++ch-t$Jbdy+{II=2I$nO!#?_Y=tig%Bq&|DAm4=T|?dyz;FiNA7%C{Msc* zBoVRCY;METTP@YIx`r?AK|CJ&K{{}jANielJo+;?G5L>os3(Z@_X9;Q*w_(F-9>Vl zFL{n&cht^e^v~$)r@D=X7gq4p86h_68M-bdoVe2`G|-Ky*%#* zBwXgaQ0AXd760pjSO5Jp|6=otqexcdE?gamp=6>uD-KV4yG5ZEZ|9O6nvKUt#Mc<& zdXsY1;JlLaxa2XZBQZUHzO|ZVdfxor?OfE7c^7DT|NfbO@eZ%dij{S{Vc7h8oB!sl z{omf^%GTkW& zM`QXtU6GzIW0y~%U>0lhkR+#JxMYml)MOJ}kcj`q-gkyIxvkp@E@Y`FOA!HSN|i3% zP*nt^OB3lxuR%f+La{7GT4(~&rT5-DiV#}p9Rdmg1Oky3gi!88*IMVC{cP5DKX>0> z_x$to5ktN?=Uc`*#+dO4LUC9soJ&(bJUP4lOK>3FR-Lkg3gUoG=KswBS*UxuUnD~w zO|(x6x`~%gkh-rkE;KtIQ=OJO-th7;Gca;3-DNVCm6cs@XJ=HzHAph$rc4)>1WXp% zjmatM?=ugk_9ZZEvJOr&%1)g=T5_)Qw@+!$jWd*Ee0?uz^+qv4OjI@NJi8D2Z zr8&h&i`ETIi?whJ#mpt~-?)xT3RJ|m>&YQDAIQeaY*Ndr3Z4a)cBxQ^ZVx%@H5Huc zPlDquYMb_EL##S|r#%hNOMJarU14OF`>sBSj7H?1k+1gyt%bx|&vuT?c;3!vz6gZ~ zq+;+Dzc!KuW2TocU#_=>K3i{%E0mka(H)o9Ygx5Dh1V@?xDF-;FASWU%J~jAAm4%=id(#SEtX6CdI9%V9N%& zRZqb^1|yp1c47unCI){IS~6;Io{5Ek7w3IZ*}%EdQ}yw>jmG`^zoC)gtbyq&weuxl zmwpj5`sZEn31TCX=bnrjLnh?jnA1DEafO)fTIiPNE)8yBzQI%VEVy7(Ouq*0RtYYx ztiY3E^m)A4c{z#Q1M8%v|KP9Urv?x{MiAFzj|}(asr>$XZWK>x?{F0 z?t!iY@DW3nrYl!LUHX!<~a$|6!hE*|R_@2_ffam!7g= zsGo)^QZvy+owyW=!cd34Q>12tM4fmuhq_c`MRW63?SdZJ{f8W{6cHSciOc-yPF{Kt zkQ{DyCjO~CQ}febv_ITvFaG77#55Mf<%sLxgPIa}W=vbXcX}zOskX=Cmd+o{t zU8zJt@kj6BWrBQvvwa)L$1+^u)idj2>)+av*ZY-vYHvhQar0xqL6eHBYBuaT{74TM!b zL#P8EXr@!^9d-wG@nqbtONO?T<+!)5b6Kf~u!C*#D3Y`DN`7(B*Ar^k54u)>SdH#C zl-m7r60+HQ7=d=5K5)uh9x05Ns6n$>#rs4JHdcfQYkDQ4RZw61ccN(Ll6)V?yE_j> zx~5wMKIjw4!GS|2!@$VTGTyp!mLaw}QlOR|5}Obb*B6#d%epZUnz}<%O>+%Wquyae z3(6xa+A?4Saf+w-<18!kBkyf}CB$td?zJcF{KZP4!{LuY`xT`gz?7lQmy4~=7~k-1 zxGXD0S%j-zFj^nX^2h(qn?Ek3vRGy|6T1Zt!d9S)L8-^aX#;aSj)`yza+MYg6Z9g$oUTU}1H7rE(Ndf%6Pi2dcRN(KGcVQVmLKzY!hrl_?` zdY*nIFBBfxp?#1L1b3R;HQ=V!X-dSV#fjK(09Z1zuwhqub=9dBou^hviWxOiE3&l3 znWMU-L<{UH3%2j&3-$yYwBD+Q&6#&#s4zX!zR_+gg@JatmyMnroYSp!3l_5XaxHcU zqYNX)r=&0uf;YgjbzJk?OOiZW8;edaD(dBZa5x)6^57Za^Ke{#kk`&!CQPkI9BUZw zA?HwCuTM@uZ3CpjqNRfpkBrUnAwRdu?pVKXiMe0Uu+iH^8L<}J*7xnTBHdM*sC$=Q zpTbbTeyd2w0*Si!X6O_+W09iw#lacVGEg8@LMYe}mp`LBNvYEh8dr_jCi|<+O?M+< z?=Ww}*>R#ZpwPVAkx7i#RFgDZZ77ZQ+D41^uOPZr1FGh}d=xI!r5KS|dRsN`1kRPW zo3cSF!#bJIZisLGc&Bsd8Ve1NqVMg`1a5bs@QKQ0x>)JVQz&K0^D2Vi9MH)qF4s7iN zR_9Ur6SMcZMhcA`Ra*CJGTWC6WFOoGeT;l)j;?7GpRr|#%~C1fSR>8aa-no5%5Z!Pr6!#p<y1>|7T{IKgf(;n9I0qx)9FxFmoSjhi$nbt4)Pw0 zb1^Lpd3RU)e772V=2qb~Eey{`j%lHu+C5XA^b5LFl=~t_w|@aeke%0yunu7m)aZr~ zysFbX=~DuRedDCnhV@I+7~Mw0+@9@<qu>8M0hI4-cWHHOHVP-Twoe2 zakX=)_jKxe8LfysU)i=`?8PaGt8b3AnG;y-ZBTN76cm@=aM}}6*Tk`p>!dlPJ>3Vl zOO{C^gi3rqos^mTdMJK}0q*s>Zm*-O%{E5LHEMr1wuWkAdIC(sbPRlgx%KMq?`~Gh z&+~ob86WNh(c42^u#g-Q8|}aPy6wJBR~H^*ZOL1c$|P@C+*OhJRI)Vj_7v@iY3RTN zzZ-1w(FKeSh8nZ!7L-8e{_vMAN}5x`)!&>V%<4#$)`&r)hur?HU6!bp(@1(PzO`Eh zZghan_fHHMbzJE;?3NY#SFG>(aIt*2~JGlihTH;56ny%^sTNHs>$|Ixvm=%3@(zdhy% z^(TaaXnBPEpUaQaBl!}SBCjKumK!rvs~_R)E52<9-X5wSKCAr})GO%h$|vR&;p30uyZMrBqeA(!H)>`d}sUn=rlUz8sfBDl-Pk$n+; zh>%#PUYsb*^xEtr?b{kigts)DW*oC>380+IR;RGYOmM510A-pg5XN)WUiWS3nCtU_ z8h`EW%6~Jq($sqw{FeDRyAq|)QvKQ?i_D~%i<)e#?Pq3XuIppX}k{;#fSko=B4GM zABCeqI(SGifk^jpJFK zvbM5=;W}qmGv(ZbKxbih=c{1=s!7VN>r%E>cDRAzf~oqp`;boD@JfDr)$*P=jXQa5 zfmlhBfsebhF*=&q&DGgsG*lggjv3a#G?j_VmyPzgm))nwIuC-0g$Dv+omlb975cUM zqD?&!VB}MpQR+Kzu<|?jI-I#Whde5=L%|NVj+%|{ziux@4m+|AhD=cfrB9cm_ILKX95q!l^L&=yrGpZrTVlR5G*Y%; z6eVDBCVeHC$Nck$#*`JldW>Rqu_61;;sS0iN?%-VMBLas<*B4wlI^PPDS1!nmzqpDlI~_bq1o`X1dzy%5m*KYDsapt|M`SCilORbadhwna z<%7Zse)}ASBmE45zMH!ojxqoIfTJMg*jyfGVv=30nGOX zXr^Y;tutYmJw4G*#UPdKcMq9Alc~T$L`a%@5`9u^j+m^J>h^4ibde?f5=|xJv87uj zaZD9?Oy1M|nka{~Zs7^`sRtC^EW5EbDxZ>9Bjo}HYp13ZfRP#0U;r#4F>W4C8Mc@hgRbP6sl1GuSY_t{ZL+^|fyL~6xWU*h4t zpX_-c@rvo z<~5SNNId#?Ri2-rbo1{XPWJbWlrf3BxYWxVxAp@wAHvS z1CuYVAD`jVqz~uD<&@UwnoFr?$_9&LYJ7c5hA<3pn+U zRylAV2Q5HJ1e_QJbcqOk!*{3m^G-K55>{y!-Q*YE$Se)6J*fWlT*>^y$4rZL!P%K{ z)z}-&v72xzJAn{}^)+Paio=ntHWo=|HKLSTvuJKS!9KfcuhQzRUOYA)vDN<3^9Q=a zz#*l>UTNNLs#_^_kZM?IKg9$ZUPj4nu@M8Vk#c)3fOQS$E%^r2gaO&`5Oyw3tUPMx ztWRJCj~QxS;L)B7h?I5qqebth+)TDkuGoBS+`;?y?|18`I8fBzt9)87yFr^~gqFh> zH}0(qst;^1x(@Q=^SI`U!v# z6xVe~u-&t}#ghPQ8O8{dV+f4%iX+g36qzTFd+$TAz08>U?dd4eQ~2VM+%X?HX~^wC zKdYrwnNu> zLIyu^ zpY5#wa{POeAB*YZ*h856NTI6W5ujt>>NAPQi%xL^qYks4lmZcM{ffJZM*v9uu7-hl z`Qy=KzfKiVG}F-eo4tun;MP5qz;>-;y4o}xxan@Sj3W!Ow3qD{-6qiM?U{@Ug;U~E z2$x%p?_roQ2X_{mPZ3f^wXh|X{%bL3=Sifg$B~D{qQ@!PvR|?m&pYC5>e$>YNg0U1AolE%I zc(5CxRmm1TW)U4Q{5#)rx;K(`#?n%UF(y|^Omyv|utQ$E&AcomS4aHb;EVX@&xv&( zp)d`$LyC~LT^23v6Vjet_W)H{v5ey!9NL#&w@YW`o z((vD(7bnoY2-D0_4}Zj&@X4MT`N=%eHVyMuk-c`}x=BxgV=unDoVx7e#gJ$^irQZs zcp(1;D(NY`x~<5y^RS)2`kemVqiiQOr%6;Rwk<=xBdty`s+Ct2r;;2nFqBy|<=T^9 z@?_1FJ*2F$xO7l#!=7N~IMllGBQkmvKb#2Ut1-AT=*SJn7Dv}Rpkjsbd8^Hg6H;g4 z#pm<>2~XxbLLo%bUrp|TT#>{C4cK)ugQlL=v5V6U!xpUH5vgOqJlCo78meCgcrkUA zwT8V6y1EY^UO!1n7IsDWd{_znugX(s_AydGmf~WDUy!LNeVlmG))%MWun7Z` zQ5!&d()``x7_jcdv!c`X<7Mgqa<)tw?iP;@OicVfyOXc(@_a8NSi1gI448CsTB%A4 z2>nH78HlFa?*uHGL#&E~z(~r^z69_Se>c2)L~(OtQ!^!V^8gF!8~HxFWSA3J2E&1_ zj^<_Sw!YkW$bMUIJa<3MX-Gz@m8vmplkaGBT`IcPW)S$->n|pS#J&wUc-y8v-Bre5 zZmq{O)hdISnp{thJ{n$vpZL4s=r3^oUoJlA=^9v29!&qU6#uXE z|K->FO$Dixp%q|ebniYZSp=QZi8D=>0%W)D@xHo;;0-12{heW~=c`8YgeRup%MuIGy<%xQ}r>ky>xzlr=ypD_$y( zR~CV;D0}F(5~6ymd$_NSU%!ME7fp;04_?nE88N-t&vry8ymb@YF!A5WWuL*nv-%89 z`^M;`&fiVuf75z)j&`_nXCW~X_U#ftG){NRF~pMm#dR2!d-c98J54>@OySP^QB}mj z+}(fY)PrWoIDB5>!f`?M;%R_R++$5!KBC6~@bQ1r;|{z3JlKF9r!4EMjLCAeG$n@# z@l)>ehov^e*hseZmffuj_M@w>o>SsGGrGZQ(@k+qNp$Nf#SsgEN~=Jl{QA)QZ&?Fc zVq+7nGNstDc~}qTtr_+^nFb)L8=7g~*Mo*Ek~Z`Hny4dBto8V?jRt z_4gwWlov_NG^eD25F0~{peuE~e8hqXG0gWA1~^kB2xINl{>KJ7qF?TC-LJ6MJwhV5 z-!@*Ig8+_NII=NVLTIB$DngzOscgGj$|tailpK&}MJfmGmP*4mk%sh<|IXQ)1C1^q zF_sK;i?@%L+i{roM2ntT@IQv5fvJAVX#^afp4(3rsXxej7^d-p82?7%zyXsSY&(B}tBg z{gN=(9BLNz%()0I&_x70fBrTCa;y9OwKjgzHih#SI7}EPceHdXEiJ+ytUicH6=#po z4@+)etYV~v)Km&ovqED38I@o?Uzh~D6LoVT@R%jJ8tOoFL_fkh9fFPVtJOnMm=O1G z zJ}^2BqIPV;H#j=}TsElLuf56?UA^-dYwH_VnwezIaleQQPksN{nk<-3xJY2~4KbeV z#VId$gFqtkuhX9+m?5uZuF?TJ1nNDy_8Z@6)yT(~xA~=jbg5s0r8+hO7?WH#5f$?I z&uyfC72sg3w}{{cX~H4RGJ+~MWa1)H0SAdJkW9FU462AY<(JAY1RHGXc&vG5RFLe- zSF{t)DTyXuKeZ!z@UzFjXyU&yN1MaUDnLZU+qah(ANxwZC;fk z5iEm1u3QOfEo!@evh!00#`^KiD>BUT=kmxz)1OK2B~Hcf!8Jt^p|1?wuaUfq^| zRV~`ndZC@4Hm7=;9Wp;90Rt0L=8IlB#0G(%pD>0V8xg*0G!+DlxQ~0VgF$;fNX#7i zGG>j(Ocp^Pre^-0nSUh0;*U&jXGVkld3*qYK5+ZjrHc?S5Tl=EYo@<^+ea30Th$Rc z+V12hTVSl-lxXKv8iFjhc9Vk2%OjxaT~K+1<-)9+3+Bs%j!8FjYp94AGqXH>{+``3 zNX>d%wk5VhoB<;nxRnny#}ODkc-{q!a%tzVoSg?i42UI)42L4?yayS+QW=A`&6q?R||P%WU^dePjka`ae6Lk7dgPhktqjFkq}rId{#N zu{6%6V`Ok&%pXpA$E2u%(2vMURg{88+eTHi9ZMrW(=1(X2j%M^FXpmn1TSeMYNB%FG(Vrv?|9fG*6BX>1 z2~-HG7ov`jw6aYoZ&y$gc#DD|W3P#jpDZHQid@uc5`|T$h-=Add z8ea#j1guzPTdB-MdyO#Hz?EMi87)?KtptRaUX_vFM%YkXDLx7N>tVOyq`;=+^B}UP zbb=qK#}kfboe~~@--G}3ukvqY98uBngv*zT#AP4K9h+CFhyVb@VIsPT2C-1)MIYT| zi0yNl(Ri|RWG&61ZG9(WpBc9GHQxUx=j~sdvHu8J``;`Dk03ieP&{6GM00#~_6uNA zfaBlmev%wTc23UuAtvjzrHQkE(2YyvGEDM6%HG2}$xqSJaEg$w^1mVp1w1tMFX(3( zur^s|93(JbP&~$-)nJj!{xih*|2*RLzcJ-M7*h%X(=y(>As)W^vKvjQ@3ROj-TRQG zf%)VU0TZ=o=2~n?Zdc4kGGJA2#wQq+pUhEO9;OeVcihNOIT}7-*SNoV`xLJn3jzQ3 z8z6Lk&j1|%;SBJ>jtD+7u^K*J-tDwnFuLAFmC;U-`ie@Qv4WCM;;_b5kz{>q#JD5D zeLAXifN(^JtF4)iUVmaYQVUv4sjB@qs&K_P}$SH z@;(mrOZ?-=Q8?hm*Lt}Lt<^V)A@g@2waBZ@*xh+s9Of<8B9_6Au5>xh76$;L{`N)( zP;#}!e;2ZS2F_ytO~Uhv8EK#J5(M~eGpm%R-djuHUbJDJZo`RAwGeNOsXd(C>B^Tj z>8DA+z1h%EVY}hcyt&TkXyAw-?>|)GCThgq8O0Y|(Ag3g#iOTy7CW~hgPu(Pegj(d zO>X3ID3ZbHAAmJi$E4^?vhFIX)afp)>y<5ZTjAATs980d4>CV2j<>#4gtYw{YsXKA zBl;I`P6DV!5ea(jdpnH~_XM3zwo657aZ<8!Av83ueEJo6WZvd!*jn5HzGhJ{F)~vw z1fXO=Om5oevIB%OO3qSW1sCMS&MI_q?yTG8OfcvZ=2vzp_N}#`G(g}Z!m8iun*M} zYE5S zL2wsuOpDA5hDbog?OgJG(1}WzkbOW!0QbPeUvqbD>@mV|o9J*2DFbC|tc~NL>r_t* zKHV(4RV%$?b-g7zse@VO)EFUXuWx4_3Y#oue>l6ZFNU6c03;L4YC*tZ2wSX^=6~1n zweQMSw8czXao-joa>3s3P<6&c9bO}4> zwbwd{@t>O$-<&+31*H;*=G0a^bGCE5c?Zea51lMS7;bvcQMzJX&r%(55U{1jE>&Q} z+OWU7Ibf%-Jpq4Mcs6B%iBD_IGj~|UVx^`$Ok1O$|LmDFn9qI;6xwYGUzW9VultOC zeeW;Qk{@`f2yEYvoa0jiA16oqOsetS9D*h-qMgNQ7L{M*H zpn1WGKWO3XH7c`C4)RGo|7JG-#* z6?pEU8mwO-Q!eoRkd%I<4X=LQbEhpz4wFi%Jn49(#70ml1`pjHjT_2a=4)JkE*wa( z&DN<5)5_5r(=#^j(K}eKGiIv2_f1lxslnSUO#8_}VAdT+VJ;8z%P?0dl)gwPfa8^3 z?5d>ML@ZnOH`KtC6RX_$^Fsh9x(5B*qQB-Ht&uf>x&t};Rx^OoHvW{Z!hy1u`udS zyvM&@RxP?;|zbhd}52hP2Epu&{9f}`YGWus_SDK~B;Oz8_ezHdx$ zwI_n7m>D_k2=`~WlGd(#c5_tQY|vtow}|}eimTORL;P9r^1B_1I~xy+Z;83uQtZDx z!50<7Yf|WpDu)?%R^V?G);5H3>o*9wwGDTzskI*vbRv8Yba{Yxy#GWi@iDW2(9=Kq zu%6#tF5}ZJw5D_G3koy|Be1h~3WRNmzH3QzJs# zk6bH$LjVt5G9QFF^&Z&}O{m#TSl6)b_r}wW?aZigrR5#tYTK92BX~0GK!%!sLCB>D zM{p-+y-;pUDWG?uCy5ZI(w2AOa1O7J^Hc-|4Ow^-dO8d zi@uqc{WU6ra90H+H_$XFd*fL-G&63L(2mu(=5p(nw8hGL$JcG)Y5TUajR1DG(38Xt zuCU=9cG(@&!J1_;?TD=CjZXB4Z{Jz;auy|Qec@-8)pcEZl56nzy<6Nw%@hmJ%NZ7X zw-h|qY&zHyHEuU_pJoI6EH|CPqt}dcHTH!ZR7IGov46G`Za$3EF@F1)mg^XB;hD=H zTb2GZH-0Wz_p+07l^v|>np@O)*;l&HQESk4e&i=zF)+6{J=_B2FmZqOMjqJqHeV3= zi6DYh%{M>Tn~tyLtuWBw_sAi}1I30Ka8@j<@2SjBStljk=Bqg5h_F%wWc@i@qmIJB z{1$eFm0nYL+#=dA-OP^{wrgXhc>8vC)qa6qxkiEG&_k_0@0`MWMBvV?J(dK=>f(n3 z3mJ9S#aGHo^9-sDVk_-U9Qm$a|4n<;sK$jxT`;W@_qbj%ur5}`2Nmm^rXFDzu!5`x z3%ZX?YKV>z%p6mR6@p&tQ1`wmA^Tq1>%HZ|#y8^mG{~W7V4cy17_`SC>EX`@b6hLp zwFw4xBHJbS45!(oaE>0;AR(o1k}!MiuTyy~rP>g=c$LLMIF^^XO5d zBdI#U0{#)4QV1G+Y(a6HIbVGM`wXUnrY~Kt{39k;YR6<_nT$m%mRX{WVCnT)0bAOA zQpU%{U0#y!z=dSAzcpQ;pXHT@r$kxhUo<;GgalkpTRahj2}xXyZf2H`zOQR7M+Coo z|74v~LCXn0DnXYBn3l9ztQ$R*V9g5ObOI#r;e}e&2~%;h&u1cFWY-+KRsHFlWyz`}+D^o)_kvlUb!xrka^gI|z`ZApDd8}dj41OJ4@{(pdRKGI9Q|Nddh;9Qb zl<-WtM76fl6)N)8dnLIm1-5Q@eaB^LbTD}9VKw6l5tgptKF!9J{gq1G!TL8dGwewA znNeJcPxdF@VLqz64M4Q5wK$eqSbq~_;TRzgd-I5WLkqnM`(&@(MY<#QqSM5~2IeeQ)LPeO=qXVc0;*nc7z8Xy4UY0ic3UoC%@gi;^s_&m) z<^N8(HUvRjdu`45?BV0paihfTWxkS6+dKH>@PKyxN?Sf1BXuNnB6;>)pHCRQsH58p z1O39V`YPBQj7AK5t+r@qCXkm(c)%Uhm$g-fD3OAI_d_}gNIJqM;C^NYpxuBO(|Uhr zkJ1ioE?vftHziaK902EpEtxOo++RG(Y(tTdjFCYE13>U z?+`GXAN+F8L%uYRZ@rg$`6C2!0y6(QB$xDBbKd&A{Q~B#;8GsLYr4{oIAFj{jF;(< z;t2^kzJAn;kONw(0M3lVjlz#4yI~0ME282= zWzV)UPcTi(htO~wWqNM8M)!v5xOn%~mwj`YNJRKT1~h43vPPgiP>Tecqx2_-y*`>- z*SyDoaV)Z6ZY>I42q);GwfTMaMq-5z+jwiSoZZCu5{u4`K`Y~uE4VKk^Kpe8%`&IP z#cyIhHJkS~CA(eo+GAi=l7$VR9FC}&!Fiv=kE?a~f7Ci5pBokmj9`Fyt1H-nty-)F z3~k`-T5Vqp&01Io5w3c$d#bpNA6`*CE=V)4C{~NShWraInn9|QU|nhBdHkZhA3*EG z>-{D$4<|*f?b>b}DUb>kAX0mq4|QvHq)(IXu68{-N-}*8Vh*VZclJJ{(aChhZ)q|k z^C+zLPnf@Tor$M=O6UB6ZuG_XstE#6zDk#BL!eF|URVv_`Ln8o?;}jtRWvfE(p$z` zFBetXjNG}*yI1;nG^|9q0uTo_n z%XoWqjY>g7PJvd5rD>8NC_z>ym{UK$DHCLfjbynLw1S}yiq{AZC(~T~#fld4m;KEc zI^)bJd|N_(_(1F>F&?WyYKk)meD*r~Y;I%W_}{dMww{p=hiU^)4{~WQc3D|l z5jRI;+xay!I|jb9K1s%=L|*npgb-;T_DVOEy~Fq^xtW6Kf^<)YVvEkpV4p%8@srMc zWBI16;9+S;E~PLPAt850k+N-j8il%-;g5JPb{rV=hmlXzgKUlT{E!JRbof<>hXtE{ldhX>0GTT!!wvE2T~=BS z8n%ej!=5m*2S)93N{DvzlEy^vf@w(?!EJNj+0QB+Yprtdd-+_not1Xtt>?bQcQ|17 zY8;XI12(iH{Ap*|=BPDh;^Qy>2$%f)GTL=MD%;~N-SF-p+;vmFj1upc{tz}A&Rp*! zDmxmIca;jLy;!X~bK1P)gm0rluQAU(-On9scJB^lD04QgJFesj{R22w+zgdDAMu|2 zd~{=8h^m8~_wTk?9Y1`+9LQ|ZF?`Aq>~iNq&|26nZD73uP~EYYUu|cB@gk6KAc)R#zkl3*gGlj&PMFyV8KoQMFy*J=soEfhsu?k|%gD1I zWO-D|T$8WVn6Ch{=h8l!7#j$FRa-_?JSUcWOiL5bNs!LgtGZXvX!p!(eXGVaGku`q z;X-Ks4IX6Z+C0{!jo)feS3K08CQ)PIo*jOR`C{V6rVma}u!K*gO2ysct)+)3zdTyP zI!Th~C@08+krYRG?cAcKvHw)6j=Z5YdfRcf3YS#d_n76ly>a-*`|&h3a2wO^FF>h) zJT4eW8JFav$Ft}oueP(43xv)L=da%00bnXmf4l|HeRCJ%= z%B0$qCjph_P2unAFuU1C9??ORrVJ-m}C}0sD^mA#>#W+c) zy8`J-V9)DrbL+cgHsm0xGMD~>Gqg;{>d!Oo*mV|^SSwj52q_h*PLzca7o7y9=85QH z85Vi7R42(B6oS~B70~`QtDL8j>w;q*@9s)BCO9Y?i15o(B&@qC9}Vgqj)sUB%bhId zkg$XOQ}W~$=8L9_<6CS5A8!}dVKv!FBE2 zRi0EiM0AVNWd(p^&Q`SWZmvk)w;h%+%8(Gh-5Sj3;5crT7)5LLd-*aaZ#g5r>O3EmA{D(c^3VLgs zI1!B#ZlEk8`Xu1?8e?rQR6uHJ`YJo1(@!{QX?IdXm~5iQoWfaouBT!W?(A=q^f%0IIZc>SaX!eiuAH#ROjsE@@+bk$ z4jr5UzN5QO`sa#O4C}!c{9#qJb|LcSEdlPZ>%X0AeZ~6wYvG zqc)v}=vVLsHJ!WeJ;bs&d_VA{V8Hw{K5ARgn+wl~Uwlu+P!mBc@AOSb;iald2D7Sxv(FT+#B}V6qxeoG z)qiI{JkmhO8$WqR=dHf=`-0HUu=~vLa#`>)bw3F>>>N(2s)3-YiXVFMHvV%WsZMy# zuy3HRE~`@hp}-DhMN&K4!rS)A4^@T0f2gyj+^*Eq_UP1;-uin5$Di};W%T{>r73Vb z_!HeqXo8Tni^U`%b=!M?u4+~_+&`kVBdy(Rwjy2cIyd*5Bfoc#1*7YKD0stiJc=E< zcC~_7Lb=-bLyKF^#gfh5B(4Pp6~VmFPZ(gsh3Z=Q{5`cB%$Kj z`SzAiDxG~Tp*G>Sp3+gg`^8>c%8EK8seLEjtce$5{Aw9=&CVu2N4q%OM&H0Ma%fa* zP{xQklmvNrzNyCp1>PH?eTE8S*x5YMJHGKgd zu#4-#MU5Y(UmUWDvz5L*ZE1?QExKj4hxy{?X&DlOqj@!F1#4J%JC@7j1$RDzXor9L zYu&FOeHR$8pp*h3m8z87pJl{|AW+g)eZ(U=wV?IkM1&! zPPg-q?#RliAujlv__8BS7Bc13fc0Z2U31tYk)H2U%r$`X{e+Dm!v?hKTS{lz?R~Zq zBm;a502+qOIK7qy0M)Z(Xs<_mdrJPsqo-p00?hTYC56d<88k zZd<86IHC+B_KR znr*mD2QJbe-+9t)zk?kJvFIGmKfSF7{;GB**^hy3+G|7F<8a^I91OcpPq69hf_wwp zigdM@%ZUUal%P4f#p(Vc;4?O)G1g(- z+LaCTqA71!F?A_p9uA3Hg;t7A`HYF=2 zJ!>lec&eJJ!{!j8X_P3eDRsCvFsi~~yjJfYlT zw)GrfBrzWGAu`faoF&RB2xQ_mGrLu0L zg(tR1%ol`?p;@${XZjl+{p-LLgBOq%c!$+T8b;6OQHn`33Od!@HT!i#Ocb)TSIZI; z+3hNeXulLuIPJYHiNTh3M>gslGCMvzlv3I%UoN0bS6YCs52#6ZIBT-&kE~{Bp=K)< z4MJ)T`7XJ?lQ4YjEV&8{+F;#d3!U#t8BO=a_w<*igAOCLhZ{e;Hn_kro`YeO%SH-I zjkH6qPMhk&`M$>S*DTJQ1-k6KL+Sl>=jG{rf7}*}IIuR4XisZ2TENvQ@5K+)x4M+5 z8NZOkC^jAORv!Kw7csH5_VZNYb|-*wE>`+&EsrSp#pzLUjn{N=iaEMA2Rk`v_c=Eb zZI6-5%!;r)yslYO7WBj`FY7+@Ke}ZOtrh&~m6>y+2+YxeDMaJl2W#kyRewNU3b$|H z9yDtYZ|Scv+%Rnndfa=Pw6lpQq)k^?L=jf7k=<+zfn{)q&7%tAGHQMFGHS{}YZ*U*{ z?N`#fZ?=?7YLPZsjhWEua_{1#_fEkFhP@CYHlJ7QS7fM17ORY4Ym_jvT2&|C5YC7- z3b{@}O}46;J_rM(hN+x(NMinG^>VjSN`QXS_{uL_1 zwna*J%}8qZ;aOHOBdG#sXGu1~cI#78&rU4)G*_64sC)tPqQdNzH*b=6#hqVoIR@sy zSLjC^ho-sGW>xaRZ0!)CpJe?kDcfI5pi=dZO;FwHQrZ)L2Hx@VdmrWWb7>SkTbnFH z=!UJyMh9{}OuDgVQ%n72DR-&;kskx^pPoCbJHJz?h7&yTRN9ZFyjx3;hOSArNULW`aQhAg7)OSV07VmmQTz^PNDdN-#g#WFm> z43))5=p^dcSmcjL?}ut*;v5eF8QDES)gBb`()4|nh!WQZHD5RS;&$Hd71;GUy4NN@yrQlnKbexr`A!5V3wPlHh zy;HMO6H4*w#tTm{Uk-xZfVw^Ja5~Sy{r(|&j*oheZc~9-61MF}crl;-b=mWeJrwa{ z4@DGCI8>t3jHlnjnuvz5Y7QDxtdvE6ARIkkRMEgD;73rQS7vAgK_$_~`%3!?+H+Jl z*Z6UwD9#Epy^G=17RvXzEb3r1Qe#{smY$-DFbLt`2y}3X`26CwSOhXwKaDZ=Ve^6B_U_H4EVH!3 z{TFk8xGV;(XX&2xlaTnZ`!Xf}2woqacF<3qFg5nfd>^V6XTr`A8q<2v{ESAz@O8+s)fAYvpGe7VFqz$MG_uyMADl+dO^ z4=wkF@KroD9Fgh^bMcX~oO$Bk`rEuK6sEOMxPc45Ya}*831PdaT)zr1#fYPJ5m<6z zR%>fJD43DyfE?KPPQq!=Wz1tC!2)D`Jp1)34wI2;hN0K42gcX8Xy%a*Do1AjG*orp z{=#PLz2$LfA(_;#J=u?rm#jyibg7Ls!&ztWZFu@&f{8E6= z*_9HVV_lxQO+%RKoJYllGXy1NE7%B>+y*ht#kYIW$V_a^>X#P@7=!PL?cL8Mw$(Ls zFYdq(J2sL~+I;>WJmBzCj`kx8=MNj1Wl?vo5S#{Fmj|Pxc*fh{xO!G{vL(C9d_#hP z&BMlQ?I%)VyK6IHekO-KQ{GJqDz4|~q>5@cjY!g!9NlDPxqrPEHuZ=oYzj{GyX-B` zu!4Sggpq8^b0*oEr!I^Axo`Gie8%hD=2q99kv!X5*sq7Ks=y*7+R19!tyH55A9Bn1 zo%=drwsET3k+`Dfb54%^IK^R?`TJT{J%Yoh(@+V5{bia_-zQC+wwBs_)8@V#^H!PN z`4$^dDp5g-0^p+BXV7R~{kw-+qGE6toEeMBRH#}_#fUdK3_jfbLp- zHQMaDm8(=SHy7=?KcB3tnDl9lsSRgw(%9+9YABGx?c&c44Dqg<#y*=0wDlGpLDjD& zRVNwkEaC+VD%Wt<-SHlnfz<@e_EzIAzXd8>Wu`TdomOP8fN(NFzH-8OgJ!i-kio9X z)nO_xziu0v{Ht*+{BWPCzs$)lp)NE0+hAbNkRO~so3nIDiG;HqI`b{&!o^W^;xb{M zerN^HP%xZYTV9(WU%mi6;N7WmJ85`Y&$}2ix|X`zg!YnxG5jC)-ZQAlz3mrWBI?41 zf{1{CbO8}5igXp}O-evIC`F1Oy##^{l`f%ol%CKLkd7d|m(U?fCzJ>w1PFoi&sxv> zzR#ZRnqi+gANHAfzBzt~@y>N$?N_cTGycjD{}mS|R0T@s%aHvyF@9`ZgCnfn!0*^~ zlkqJHhVhgJSUpy;jMWiFuic5Gshl7R*$!9=uOn?;X1+V;nq(tb4fc1aIPGIrU3Pxm zU#a!1UyLB6vlgefO!a(H(bdPE!+X{QW$)BwT`_afUo2KB6L$I8xW>2ls)sE#x6V#W zc5MMIC6?;CpTdY=#3*0wS5BLPO@hP0^3VQBYWAm>e1o!aGUfL58phXeUR#b2IdNKd zJ=U(Yv#s>nXk=Do6*Yfr*%4)+5XQn6eAzEOHu;ug73Uqeu%DC9d-p@*%>}>v<6rFg zBt0DoOoM7KZnb`3mMquEIv`@@wCgPd>a8GhTx!NuZd!a0UOJ1E5s4r=eqQ{_9rC&5 z&Ni(B$S>30SaePj8YQ^lI3DG#!1aAp8=>~|PhIPI(&=cGrPI3$>9eTU%o62M`4e|K z-%OgmW!_X$)HT6^XjmJEv16p(y?@{NK685Hb5heOSBuKs5Zu&pTupAadLFt+Lk25- z_rZzndsz93zXAta=VLLxMwEkSs>NN47|Mb4=bWF?7PjeO*Do-=4Ids^=B&ppI`N$3 zt-iO|zqUQ;`wsfkSKcGU(WKO>lMQ%W^#>Xw;B_iX+V!VGuO38ms>VFn+dfo-?ff*V z6_?TH(DDC8diC$&Q`0e95pZu&s3iT*Ezdxet|T#=_=A}dgi~y1+PBmd)L$XQLF+l~ zpG~=aBfeDm5Y{v|UWVMmwp+-xP^y{NqY3E~=zt{FWY3thTQC6sJ-3~xDg}2iAA0Op zs;k82nR80D#)9L0?E_Y4=7BlTuj5r{TpO`Je9)bvPh9s+$kl@~DUur_gBUEYD@8;- z+^FcssewFYesS0BFK>r)%%Ai3iO-3kX1t%#zB=f3C5pFVbE((dq=%VYXr5KDUZYy| zxH$d~z!2~xzxaYipXZhI>B4t6!PP(h!kU|s9uCpvYl&!QOWet&X#QAt;az}`Z*5ws z#KvyUBd&<4*tF%1>A0gug(5kvg0lDd{Nmr}7hK|l@V)N{idMC1>w|6X;{;U*j((Pj z5x1tMm6FQq7rG1`q|Qq|R*AW3nL1zTG02@iPkJpsx8j^`*)tgrYfTlN*Ae#E;uMF? z#Q7R$C!`R%C+8*NGah78lVTv}a1p-{9S1F;D_oqM@6Iv_8-|=^vz`82r(x&+aU!@# zj%E2n(28^k;mP)@IG}_Tl-Q4?4shfJU!K1z6L+Sw+a{^<6t;$x0?e}RYos11^4vq> z=&rw#?t}KPxw{GyZPv46VO4(qT?xgHUcZqG5e%H%J52p4k*gh_EBF$XZ`OoxWLT1_ zno3FReIb}EMl7y!hpdP>^r^{Q*M2aB6~;_9hcfl=fvHb!HyIgS+sw376=(K$C3j)~c}gPY4kKXqo8=_@l8dvCU&Z%k}q9A#L=!y^JE zvGqC>)q5`bGY^$RvRwlhG?(`@y0V{-@fV@7#QrtxgP!mnN!61+vU&x&Q#%v;+gL$_ zc7;!_U=t$yMD9pbE^VI5KEm&>T%=t?&`)#CQP9zp*~zotm`&04Cja|Ol1!|lcTET2z^mDWkowDom+ z&ylVu^?93tKp2Ltm5F7@mwM3fXk4xTHWYcW?uv|(o`1zaA-dw54Wh86DL!l!U6Zqu zopHu)Beg(aZ{3=1Us`cV_%^~cf?fBQSf?MO| zCOXM(7siTg2eTr-9am&PS-%0|jzY_pF5UWeBy4Y)oLuq z{&`vN%}bl8QC^p&zWcVLI~>wZ>#Lc2HrJ|Qe(#lsZ;WMFA>WlidvDekceON4r&Lby z>@Jdwv4l<2Igluj?I2N%A1`g3CdlRi+7NU(IVg_;nkMZ5o%%Xee{W*z0lKzEcBno8 zu!7+2IxPkdF=tMTWDEASuX*|yw$(alKA$s{ZOUi%ZRDA(Dt|s@u({}ObQ7QtetRCV z>WOz4&91kztxdQ`Jfwc{t2W#%bVd<|Uu{oEv7vkKdhVy1*Nf22Lp`Tq znug^|*Xr|$@;-0fH#ie(9%fVZG~%@J&cofjer$SF9-kWSkdbJ<%5YjrOJS-J?pbK zpm3x=fxg=Qrt0^qCmp*{y<7d=Q(6qm-;-WAS*w0Z#hg|6SyT?%>R`{lB5&8Z?DmV$ zK8ZbS*Y25xwx3P!w+2=6;>)STu!_B zb|fx(Q?2xmV9yM|&g}a){abcqO(b~4rV?Wyz&uM{UxYrKnK&3GAkuiB4zw7!@4l@ScIUa(qKC24|X)twY{Dq(MVce9hppiW(1;`j^BT&wQqKC zdJ|yd?V!F|V!lxMOusy63OP*}b1Uq%H;aWf(JB7(M)=A@D+S(a{SbILSk=g&Zudc-!E;mG~6?m`owzGF88j~e@`i? z;(njL2hpvBy_p7m!A>6O+JMz$4J+yz7=D7>(pge2E|GHT-lo-s{OXm>&R9^4-gg7i zJ;5?NaCMVuSh+$|atHjxU&X^~SpDWFHBU*rUz_k_cd9vv$q@_Vvb!M1#`w zF+>b^vz*>9{;$Q)Jm0qXRpbKeddiu3pzUc%}EOnF#}W)%PQ4SALeIIu>|} zVdWvxUya9tO1>!x3!z;*VqCA__2UwIzv|{{s~(7NaAuB1l;q&5;`= z+asxSpfB;M1k;n)NN6G{xacX%$_w1VPMQ*xEjYcMB<*P^Q$WiHz*v+a3WK3QR^{ zAmzQx#p}wNYj3>|lJ;3H(bcC{jZMob`kcLUWz5?R=cycawUSf6M@rapT5ZMCpdvLq zsg+p$n1ZVa=|2nS)VOM4j;~1Z5ipc8JAZZ524Wr_L`Y9W2e7YRVFpL8`D1b;7ncO{ ziLh_d_g_=)1rK66y_*!zhJj{`xG^o*axXP^7gsNU7^vDKeQ~U11PW!k+GBXnIe^@7 zc^n>mg zkFz_?D0ai{7IH4vvR&@I6OZ!V`7+;K8UxU*tYao@K1BRTlG9`btx984lOH`QaN+pL z?S4P7HXIR?N4bP14i(3U@R#ShhqI839H*+dFfGY;Vz!2g=dVsW2_PK{G*U$M^`#+! zPwhaB9|)96=ecyV+AhWK$mN)PMn(wMGgsOB(Zt{z z=$7nX0oDv`QB^(>f)#RZbcy@kq6eAc>pkd4X{g3-4Z)mbbsR6xilZBY_IJCdw^X4? zB7{Zd!{u$#20m72E4F%w{>}FK1TvBU=h4hPe7-VKpPgrbOXdzF!NB4r_PneMM5 zX24|g;LO8qn6I=M4BYHMnl60ZaGD9gEr!+$iYy7%v5LDqW)wv{-xu&FI<@571xX7_ zdVSXIe7d686JA;yj38SMTJvzvK70eZEHHc3S~%tWB_n^KLhJk>m*q{a+MjLY4GsZj zx?Y|&=#Lw^>nO9hBaL_5?9!*upjh+Zr5Rmg|MCxP-(!?D>_0dQG$@GDrsw!JerO1y z?Iv}&yCZfsXdhO6bvl99ta{ttDOh{?SN{w`d(CJU-47W;nP+Pxe_R&f?gT9`dq^^u z>mA#HjLZ!6NK>BN9K{|dbg_D)k4G`$gZ$(6bY{eiU-M~*J5470SGVQQ`$g7e1z|Qk zT4MQvYMZ)bcIcL`V^#NJaw}t za1+XmhSxx_EMVd7v^J`*DXFZ!x3QIYwfm|LFJr$hoCaNUBT3V}Zpg+mK!t0)YV*(Z znbN%hm+o*r1I++!etFbiL$&9W1+~D1F{0MR9KPx9gfGKp*Yuy`=NF~@*Y4CQO*i^% zzQTkfK;qG|t@4-mIFF(~Z1UNP?&`IUxR!qRrB5ye1qOR?p4o^&ld8)6`Hd3tL@j@5 zp4Rc8TZhSWkG!4Hf+14cjwORO`IN?y+GKyq!jt$98#sl`@-|aGHXX>mUSOFgG&e}6 zH$$0+NWj)kd`?L%tg}pm^M*va}Xy@a?=)xmuz-Cw+Tlwo$|17>Zk0^97N zYlhvJX%?Hdk9BK&a80wv;#xEj3s$X*miC@d!Kzxv&pwAqSDR+Ph9nZ+BRt}7+usp) zTS_z2{)lx(uSr#nr7#j4Q-9V?Y;S%6V{VwRb=xyeFXj(hXc-xO)+uCrdu3P$S}WNB z+mG_8A*hX^m$846-`Cc|b;A!Ge=Wogs?%XLHp!n4c5%6Zw&C&r06QsDhNPK7oz zL+Iv)a&DW8`|g-$9G_adx*5;NJ0+L=bIvskt;34p^}Zptz}e?0d+SKASG9h@$*_O) z8Eq*0tLuF9pN}G^qW?TA@(A*uVEC&p{a+dky{L~;YdH8VgOj_9*yk<_JE=0Rj+zR= zES&Kv{JBif!^hVu98JTE&NBag>;1#@+WmnoIQ!HYpJtBg;kNl;9uDU3A+xzBJ`Bot zY~8cYP$R&8`k;jr)@LF4hf^V!OZ+=$ zUdnMP#h7-88f7h#plhGisGveWT*BqDL8z1NDjjK?C!_xc+Z{Vg^pE8-)5FKBQdF~^ zJonpG@aIPjVy{TU-Aux{t**yXU&hIJYI#>T-m~xIEHb)8ZPh6D4w3R?x4Ve?xMRh= z-D7YzO(E%U%I%jA-Tuk;=+A4v9-jW#u)g_aZ`xN2d9bfeKZYlswZFteH#*DXqaf(N zv9Co{?>=a#ul_Kck(R5{zzYypC$=uJoVBWQxE@+Y0-f z%X64!9TyT>b=%<(SVI%9msxlZc`+`?%Su9Tpb~6#e^8$e`jm)KPN1Hh z2lWn3&36Py@boJ9vAl6yQu{G=yqSE`-Hg9d+(r1nzkxdR$-SkVNRqBiVbspA##Jb9 zPRy7NUp}qJD$r}md#!r7H-=NwD%w(WaR67qAl4calFuHf4Mxi-Os>=GTn2g#RyfyF zKH}6}oe7`n0dwVuZEvd6VYwsvRKGCJwLItZMxGzM`P~TBFw1SOX2gU znYn0Xl)t%B%2Qe8;B?;^L(ZS$l=H8%@&7Z(OE5Yo9nG5|w6Xz?hqd;hd2u_%DPb16j|xW*c$nYlP_5JVT!nw7(l&A~Kf%$_ zCaYM!Y}HMR>DZ_{?7gXO z?$6i6uRWI@=%ZD(o{3ZBgw2Rn-TI)Sl1j4;XHji_!0)*OY1Jeg6hDH+p;Tx%RSRd+ z?W`buPY+|77r0rMG|6zp`Hx4M+x-eo;ubpVx`VA&G@oWZd?W6&p5^quBMa5U`G}$# zF{k@=n9@(qdiQI>XPO=q@< z^lAC7NGPPV@5dCJzP&LhFq1qMOpOiXBv}=-mIA zi7(kd^*d9=SFOqoYUZ+j4%z$mQpJzBBv|)c%kmO*%~%z-mcD71b#fVOJvtJ8KU-ummLS)PS(Z=dYin=vSE{-iFn>Fp!t0UJ+4$#D zuQtZ#OP}Ojhe+m+Pz@RlheYE$i5T}TN{{FZg{PHrve~%(CBiQuWp#SZ>Yu!H*tLjC z*>3djx`Ghy*jBMx!{fwEEJv^+TR(YJlZ}GJ;xe=S4R&(p_@^NtO`6>pQYiG?{%%(C z%+0&j7^K$>hv13mRr=6eP9AE@0+ zn@Ld}%>F3*E|$kiHZ=h|*z73|2z*Fu7%Z#%77dl8j=P4!-l2Dz&$q z+4>USz%u?k@geo;^5HY0pWySTJKk)+yNv>}+zxk9?_uMIrm{nF+GS_bKNllg$KEhW`*cPHE} zHojrq$jZ3ei(KwfOFvv4!=)EHJV&EamuyAeA!tS`4uv)N%83T7Sg|IRw&qfQ)o6|S zm7S%5LVKmWyMf2Ki)=(vgyZq^L7!(3_rD2|YKCGdhs5ugAs#F3mdidC8c|2qe^f`G z7qXF!N+rJr%xwP$uL1w7skqo{%7(mCzVUVUKktdm@ghcUfwpD#xvH3n16Pch6!c+F z>NtOYD3c(lMB?TrwxR0gh5df-r>P;&mj=d~WS|H2?N(+Z!ib9u;Zh3$zpdYZ1je0~ zDpRhbOHr+I)Bm8>+66ygYq0IEo<7>->nDns*+X^OuyYpG8)wwc`KHI1E^PI$9Sh(8 z5e6x-#oua=s>qlFsmpW$w1x^NHr}{w&#@?PyEz6X-inThVi9^pYOQ3p?0qLb{FSpP zJOrbtY()z;^#wk&+6bB3BF8QLDu}5S4c2K4;hYP(UqTDAY5^LFh&y6xGWtEHKrBS|2U&4x2-| zsiPVkWvpCq1P;rlj{Czg2)`2L*K$vSrC-%Po{EP{sj~p6A)#Qnpj^bJf#|{pr(hj^ zsde`!`E|c~B)Da2(e9R0-Hx8Qm<@fQ>_kpORHuU>)oS786;9A^p`z1tA#8YUF+&wG zqtD3O@>UeYL@Q!jX>WAU#~Z1HX@O9?>++ps41i+F=!WXUAQH3_!S<3JN`G8SF$- z;{M9lYETO3m!xlE0TYWGi59Gw z<%hf}bfqfwra#&CTLFEqj_F7+rbjCBTg@&F;34VNaBA0~sP}}*?dVD=ph0eeO8w#4 z7%k8xa?u}*6(YrQLEWK?rdmZ{bmJHJxyp+LF$#T^kx;I?3ReSTua`{^52x0_(XrEY z{ppfB)W5~v!vpW7QEpr2x~guREZw@e|MxhkrFja`P+)BdEko4k?bFtFmGLjm!?iLc zwSsO)@RAlnSA;=NEMZD`^63s-^03vX(lvX#J=8vyOBu5&QEccAZz!;(IA{&|w8vdk z|EMTnBdTN1#~XhF00+nMB|JsQI(X=!Dro23eQ`rS^JzFVD3s$wYZEfftSPF43iU9* zf^S~l9AU*-b?4pLQ$2<0JcYr7=Dq~lv}P~oDb;Xz7ch{-G>p5TCQ}5hI{3|D8eUjD+@THv9&P_KSQ4kN-I_Ii$)$@NMJ*KbmiIb|AyR8m~ zC&bKko3jVh=~t{;+}rI0m?TZLy(>DDloKT0*KK53PgZ%$-SL;zzLU>lC(ugy`$rx5 zGM@5zVSz~j%Fd+0;$Ls#-bc_ZW#ij<^aWGO&)*(NEaO^aVqIasV4ba^ZFak|E=jfw zZFrc>*Tktl>UPQz!x6Wi6?B_hg z75mveTear9rjf4V;hq5vXJ=aKvq}#O&#~PVOYhnt8~A*4$>%5egFf?-D~F~7o%N=| zY9%$g(@XV^x<39|y%i^Y?$MH~mU`ro`tWp&-G|HSG}&zqy5~>Fmhuoj2wjA5C`xFs zKZTAQ);`mPOE4}Acm38ksv(#HyrZ#R$^%z4h4%ReI2Y4?#KGrs9bA_8ohyA0;^^ZK zN}q-?@I-JbC1~^{kMbXyR{u&}dmnjD5`<6d(dsI6-tNcUe}rs|&yXYW6xWeT^nX6; zKhNsF$Cm#agMpg!mC5rs`rqcSzt)i9@P}Vs!f1sQHz!CoilmCARM-cst&xyOv#pI2ZoVse!n=)16jcfT6r=8`cq7r?BAMk!6 z0)iLXf!#gF4Y^aleLcb>?`_WU&LvlzQOQz^w<2x}Y?7#v^LZB!T{eS&lA>mRz@E&4 z2ULyh`S-Wc9S0=hz8-Kky#eRxb%ptDw+v*S2`po{PNuG4eDh3*rB}sWMteAwBe^3f zPE1GR7JEB8qE$J%sD~mX@FLp3Y8yAV27B88i^*yOe!^vK9Ca^PCc=+ge2?=SsUD$Q zQGNC3N9}O&7Zk*QaZiVrfPz-oXO2>fr`b0E=rNpDU=c+rsc|Ozzr7;%qnPIUt_1_o z(#X~Oou{tnswHlHO!YUI9dUB1G3U(pT))K4#`XBmuzyHkzkD=o= zD}IaKzZYtndhVmKIBus~yL}}UkKtyzbFe;gm~n^9HElf_h+ZF#w3r+jnZlaq;!M#dO* z{?m|h?bia+i42p8o1bZh7yU$xw%3TNrx}F`r7BZJ_8Kz{pNXdr5Gr53rxDXIj+d}X z0oXwI_01wZy?jGHUU+tt9m-C*E3Ozg1&Iq=ufC`)P)X|ngr2v~K!}i`-E3D??-RNi zhOAS(vLuAPj<+z#H7E_!P}5&zH~g|Rlh}}Q%i_T@$~7%wGDR~H_v!r$D1THQcYYZw?&FaYm30vjg3u+(`RdGc-AS^ zn>#c~+|Tf)N!?uO&>^=?_v7nAPriSoR;AlBvw4{dB3U`r^v#y~r@jQbIdCh$CpXs2 z2O_gPpo9K{PybWSWR*LA!?@e;S2)j3hcF6hXlKI|B>V^T4tActDT_S5cP_FkzO;RY z>x||1wr_g%*oV;_pMc>@j#e`?*==q3TSBye(Ro0t`v52=#&#pQcKA~XIc9~qm-zQS zt&SE$#><^po?V>Eegfm$_MB~TH~GL0+ouEscn5LV{#UdFnVJqr)QpGJNp6LgHQ;vx z-+uoyu&4likGfLGee-G{lURycs*pUO_vW0j1+6tbudy*OZFgM737L$+x|RaNXhs*i z4Eer&yFXB4b$qPaGu&O*LRu!a#@qBGmnwb7uZwT;V8ri{GwWpXg2&M+U@z*H^Zr@o zxLd_%8HUXmZ$JCw)RQx2(~}evk^b&VxYR+vjco*3onT3*mbR$3 zW7PM#>asQPUsk6}2L&yRA!kxWEy}gjNndmewb*`CA#}JhdgIFPwf+J`T3*Glt6^?q z6i&7L`9|(hW%mB?E`t(GHXkAY@+olmInxT??0kowP|Mljr#W-p(O>+2HhSBuLFT1I zzl4>A7!oE>{BDzdhD)*u&n9y!yAp1j7$s`H-cmQ$rvvs)-WpkI$!UZaWSw3@L(0wP zCmHUzg(OM1Kd?g;%y?Rkl`g`2q0|G575CG_$<)%6OP*lygPzqJJiznTi#&46f6G{+$t75Jhbv+PPvr%TRZ#A+m8=Ckmm&>5!(q4a+ zdg4eViAY#3?q+hM7c>D!-p4tXLojOeH;tp*>kFgfOS1sMl0MJx7C%jy?TD_&`lKTs z8Cd>Yf=uF!(Mo^U-q?)WU+Dks?DIp5uI=B&V^@R<0V5yqoHncuU->cUd`4$3gEhOmm)F;V#yY>&JMH@p zm(x!}jc1oXjD%)vT}aL=pjj8O=_!bu5qoYZXi`-^I9s=?aLjh3JUHwkVrSYm{YNvu z;NsxQxhta+qa)FtlLt*n4A);x)%o#&?F1uw+3xuI= zauo^>tJ_oo*(LXVSubWm|I4#}H{xWah<3JVPQi~1iI%VwPC(=4f*_(dkmDH(@!=|_ zm3HxGSqAjoHy-@%rq_=M`W)TjD&2f-xz=jkW_SE%XoPz5oo|R8zlt}zIeOA$Ap%#W zmeK>RNF-(joPvn*KM<#Tfr=A)+Wv(Xz4Q=Qg|~0(Pxn7%+_Nf5n+xOsysRVn)*(>s z&rAWL*4>Zu_z~EtYQ==IoI-kkX>{ViCqJ?Dd(HB>#${3KXIA?4Hy6B3t-F$L(DL}@ z6SWyov$AJa_*G+Witc5{CGPe5D}b$^#7FVc7`xe+e5PG#X>Ev^K4cUa(z9iDTngNZ zOZ_YKB`J=u&hrO1ubD#mUCp9L)_8OwPYypXT!j)$CIC?KgDthQ1JX%~NfogVj7VcK zQHd4EBSKDm4x&RW1AYB$YsUs{Ye*}*O=Ll;FE{4hRBE+{(#jq6tr8uc-UzP^nk!nfh3jXhRJeMV-l{4H`FGt_`Mu}b9C8`t@cVQ|UM#=l0_93Gr10mtcW(Z!kA zyqAO4?D!Pv>I`}01TpLKDslHwd(7?XkSpSkFAEx&12*Kgu=Mj5ij`qovUKVqJ#%p!*46N)WZ<4LD5~ zJ>N;+*(Z4NuEa?QMA~V3G1I5a5H4e)Anv!vnyZC;^49%nYnTA|>7vup%t>D!M(e^V z018s~x0u4lenY1AqzYhkXD-^9SgjSx8j!Z1T;^LKphnw2Z)?>S7+10zl{>-(>#khS z?oBEmhF;?SEd0uZS@UstZzu*QxWhc&BkS2#8eh_eLdJ&h;jR8(q-mwU(==&%4I!=N z>;lq*Y8`8RKmL(55NmPbi_>|OJEAlh!mkwYwT83S&yH?8OL*=Nc~NdM(d3jsS7Mkm z@4fQ0YG{})*{hEByc!$-`v=KKfu-NSmvF~6lRposvQk$l<((3nBE8oE$g-Ihv*Err z;t45I&gD578~2LumTnUD()WB$okESEPGZF(3_~wU`+R)M*pBCmi50bVAz8Nk`t^e0 zZcZ-;-AFKo(!+ffoPD_DUN_qk%2U6;nZ7z&X##|@Hx$DT1n`)VN5Pc7(!A@S!}s{( z2;IVg5@!*T_A`9cwKnqpXZdhg?;_9su)tWA4_cb55E+^fxcKJ%b|jGW?ybGA#S9<| z>%g@l-X~HLp1J#bE4{GJ9w2^w^8EvEX6NPp=xl`Ud2r|H0ofN|AlKEt-YQ!oo_}Me zB{qCTHxgKU>~^RBeruo3_mNGGn*pee_Fea)hVmfoPGX%=1=oN|B>EiFi1LUd^Zr77 zs2#*!Wp9fC5OqIYOnG-V4SlOUNj&ka2w#$*QC*v{{LJ@24#0*snR?~|;xx8BgWG%DRHe9oDG)IZpF zQk@u14sD$??GIO~v*PSU#>VFzgl~M9O*lOK@DviUyducM1d0Ewx^gFUzSl$&UjlrF zg;)ajBa(zzM6Mc^zFwb|{e9@&r1)3*Y|GJVviv#Y?Nb4ncf*S|jTS#$nX`_byGlbN z*AbI@ohkIKhsd36+Y!^v#xsJQS?Vj9sv{o5Y}u-b_fMX?$_x5Yg~>I!O5fd*+l5xa z9-BQA*v^=Ieriw+FR3CmB2CCr{ z=QlS~fM{7~d%sK-h(~VlYD-z&V7f`33hn(#?S{j75}6Ckw^{O$G}B;C$u})m9;|Zt z@k7Smfcy8D+*L~8WMC%o1&i>+?`%Akh!;1h3Q=hEDbkAHFA{zB*oZ#x*+5R_B|cr< z)_!vJi|tL|O57s(ez>{!;$UB#-05?wMJWwri=+%ZyYzar=v&V8!SW%ZqBWFOe`xS5 zN~%Irqz3&E!}zHz69~Vm;+nJ6nK(U?M&TS#7QZCfJOcqgBHq_hW2Ba|NKq^$NF2x( zpA)%5jwQ5n} zwlq3`m@}z)#+$2OSgi4)P&+@wJ%k|yu`ngGs|LogC(R=TW{3KEdB)rv3g^pXbZEO1 z$|GK5>0&^ar3`#ZNDEW>7BcG2Q&{^XeJ3D zeCk}xYAj!&%!(gCavy)H(iqlH9w|hF0dBD~uDB`RiapWm@321ke-PG!S|w?Kn#k#m zli@i<%|dP#mrHfOD?>hAf7)U*-xkpX3l|OhI=7Y*r4YT{sSrSrlr|MBabH&aZFj15 zNhv|Ja^r0yCy+89rkF2Pr;;iCPi$hhBB}g{s4om=lJ1y{q}vEz*$%;Iw?b3mHVJjdI@GV8pA<{3GTXBpN7^HST>xN;$xNyJwtXYhGQ4yf z00^ofcT~Vm$(-=boaUdHKljLM9<`%B?oY1S>u@$dZn86rreNf6gOkjQGo>Tb`6iC` z;?%m_ev$N#A60OI{0-m0j|74J=cE3A{jBh(0NQ=0>&!m|>mRQam4AE>oGtJq^8jS7 zA7%OiCM~AN7XBG^OmQGvFI4|=gwFz-V1E7Mt%1K|@sR)ShavgRJkr&+1>2a$<=PJ@h+Vo&QClR z8<;1eJj1!UEqTJ~u&%bmVitRhz7sBvoa;!|haK340+piZ58*?I4JdCJ^0@L6o8Fu8 z;`Rx~^#@LGO#=T3kcol*uYk<_=6vV&IuBxvyrDHe!mI$)n*d89ZyimgiOP>26`?^t zo@h8#G=l?sq&*Lze^41m{Q1@$%i)cyyH4I8B2XPC z)E==C@S@5Iobf~ZIVJnJ-gYI0(~~B7kCk)HDgxiWf3G^5#d=B&?lf2xSrdK8951?S zA20LN<;Sao^y#`(>Ieq|*QI>_EoVA$@6EU9l>Y%}4xGDX?>I&En2bN}^jMyhGxZiv zMV%Mz&oD^-GZ6bK2AV|t3tLe=)ZXm3Hb* zMlslvw$-sg%gMLJ!%GY2^1%!!pVJ51+p;AuQ_618{=TqYsC)^yJ28)|w|SPwkoIo_ z(%D~`oI@ZY-w)?qu<#X`CgcDKc?XVp{IIu)O#Az9bVVIzRPj&fieC+Rf=Lp0RlK|X zkv`~=>XOXTluJ(WsWN>PX;!ina&4l?f zCd^n-2%b#UXfgOmg=%8Cdj1$0U;wUQZQY4KwF@3-%`UFB(h^oA!9}0;o2hR0JWOH* z6)`pxzppEkWLhioc9RV8A^B8a{r&Z*1{Sbf$WV=){xCK5vFSbauC%h&xhH^+cgpzh z<>m;GK7>D^fGR0{P>UB;e9{~wfXNMa4zY9xJjTJvdetC|T>SM3cY&L{+;eiR4ga)H5jN)nPlTNVzBMoZ_ zq5gtSWEV%U6H5OLonVEj?Z(MuxaUWDldAuBtOJQejv+b*fa%iQq8aQ<42!T=l%-Ot zvd`8kdp&787*)uv5PF3tS4Wm3e?l}y)^lwLS9jnNM7DNA1jN+NhF+@=T317ak66=bXYUkN3;JyX{ z*Qjjp@qn;=epMu;Dy}C-kHtj7&4Je_bS+*GeA;HzhVb*fhowYW(pb5=P5a@(#Afit zR$Zz+K80oIzTKj(8AeyLb6r}JG!sQd%N?amYt4ij(p1GWUgYmdJOzXWHfly`PqXoi zLs1-x=MziW_iy|ft_DswiY8pLt}%IVQDx!Uu4sP!4EKqaO)~CiJjSO0hZe2tx8uNs z^zjURkI~a;;S?_hM#EM^g?!=~hEBrNUSR7b5BZI*zIUWI8YtFuDWIPpDsxIn8S7i1 z%ZQq@)*5!nUSyhS!kt30>Uhu|2uh})ZgBVdLySmJ8`x8-B#16sg7LECS-+UibRNSQ zJcpOQf5iir#SSF#V6;YSp(h#Ok<%=LP8a?`=yu@|kU_9YFP!0h^uvTSCr6B=0ghb$ z;?^ZOX}Kz1`T;yMEus9t8AX(4D18^=Ox8l|JFX0OYnQ2<(+V7fAY+AwYYPa=TB_!~ zph9e62?1gDdIg4tvRuNCD4_QqrI*?~mTiwDife6jk|C1vIBr;gAd^?Cf$=cuX$_4uc);VNVpKU4;_urjm z%y;q;FsOR;F8oTyVs~oA148b*DvNZ-pxhDlS)pcNRwu&8Ldu=^y)xw(<@${ui46y9 z6`fWyZw(}J4Vq_k)ii1WlNr&tlWhW3{ro*J^L~?|1e=L~P0PXX{!X&*w3wQEAYjQZ z-h3YNn$C>j1ph$CyU>*k`-<~E@rnYk{2IS9z(;d#9eUT_GALb;T;Yw_KE3L*cz7sF z&YXpHdsUobDOYph?GQYEL4c`4_AqJFq$?qeK_o^kaWhmyw9p8c6G-n>+!pM63+e=7 z(%-luL12cDW29 zJe}W_%~tAk74SQ~je8iJ)YFBidx@c&GGxBE!V&7g9!Eq&0epm?PW7tQqSg3k5kk~= zNA4nOmd(mMA?Qp3Jz=!wez0wPGw(C zkaWuO#pAF0>0~*l$0)cH9tvX<$4bU>CNqYerVyntV>0ZRX&f*EoA~|a^$&_yCjh^c z{zeji;bNx~r>rNcncZ$gD{vqiQ1eeQ@&edIJKs)Vx9B0mn7?_UsI8&CptU9$)GYQT z&5JSRkdYf+^D*<~K6>Rx-ol&%mGSq(A&m#gU5XENGowN!NAvb$-8?DhO? zJFB;OKxAzL*T6IVT4U-;DKvn>1-iWoznsr#*a?C4+Xz zkN+~+S=NWZ=)jeSoQK^0WKvzO3IGguynE?PB=&LLU?fjZ3?&tfN;-GhJaEmaD~90p-D<7cbI@k?dA2Ys2k6@#`0~#pEZ>1q+_+SQg37 z$Fpp`CDZ#@{||tx^~4Ovo;$RovIRzvFjiSd52irE{mpUY?p$W9X=MGOjHT|`1SefR zY=%>e`z^$U=eH3Tr5VCb6x;^`0Lk|veXhdoM*mA z1G5w5Z?EOa%$+AKdtm3w{45(@D=g><2d1Y2acJMQ(v)G)?B+t|EFJOmil??0v%6};1 zomj%Oz<;4%zmiD}S!Dcx=_u?r@La1U*!Ti#8&3s>e8)Z`_`)2gPRBB+_Dt(+R*mPx zH^4S#ymJj}Yc@uw5`D}Vhn%?OWv{aV1I);Acf@iXD4R>I=a5Vg)Pv3y8GI_R?rxWK zsu!$3*e-pG-+xFzw*EpeDI;dI{*@-GWMj4RmrGMfA z9uWQ7{p1ccRQdEkJ|Ew8diC}Ld)3O4Xsf1o?gRM2^TYeVp;84nR5F91CfXFX4B5?4 z)wCx8lgZ96%e491O(|bZY-McUcIzPxRSExc%d&@4>Ci3mG{va=fDK%cFWJN1ajvIY#Y|dueN<+=gAiy9V?`x) zrbx%`vqC*tc;uZ4)cc*86elmJU()?`xi3y~BSc7OBaHC;5}odcAZ`|+jUWO8-A1Ei z&_&vQ9!Bg#KeZ4U7OzuI^63<>)2^}qS-(6Sk9I8W?CH2EDMCsa>C8k{S5;w5Nc!Ea z38g)E>-*{%0$`34y7Y!~KH_KA#!gWc*~mvxUAT7dtdSo-eBypkTq9O~8Af1PnrcApg4ceUxf$nt&-EM^JIUcAi|JzyOwn6R%J15qP z!?oeB4s##0O-}u<-{kM_CVI0d?GLESNSS_kScsXIrG4p7@-Cs>OoyoiuJTh#YZkFUX{L9T`vxz#p*}kxrW(vCXA$qSMDW&t| zeznIkqkq-Un62@&#swDI(}})OM1#>kpd0=YJ9Clo0S%EQ?ymd)n3gC@|3&+4YBo>gy<*|K4?PA}=ej?~i*4<%XShtut(K!N z#(UL$uDE!*iYc@D3fKHxaI7S@O1hwGuOwqh?tlNuU*2P&P(LwB+*)ShcxM(-Fh+NL zRk?J+q1Nv#i)hA|^;ic>XKbx!mSnjYPs4Pb-}@`>Q>jnexvJ$ePmT&!E+HOU1jq=z zsGoYeIj4-*)xWuF2b z;S=LemYkm}Rr!RUvlOT6LMWsMd2Fr`opU|H8IG`ngwH<@tIqlupPs1S!+{4xAvk2dhk;@%D$HTB~Lbj;ou?U%!TxP78 z%e(O#a>L3|*4*G7!Md_%ar4Y+89De@Ik*n=E;b4_&du&)32VDWTByR>JZ2mACC?{b z*Zy`R3P{!Q*GS*Fkz|?_JpcJHTXknrjCE^@a!7(n#j<2DoW%SoEeo}uLhN_|KG*W8 zfV=+?!j@{})c!HlWXXnn>S;t%y6y2P;}`vQ@M7~%7tAHJ`wNf?=6o@|&1;zk#Z{w+ zvjVa_6Z9tZ9ZuN~J(QZ1^s5}H37w4AqGiitjeRuKq|cPi2hDz3+Q(lR1DkF&n!~?J zvkOdsiLh^2=|Oc_+3DHj+w6BLKPU>Cc%@bw&$B)n36ULgov#Rz@W(0}@W$J5x7}0j z8JtoP%Gl2dz2}Yx@3_UQ@3$X0xa`rCpc>8vCx6y<=^CAk1|2#o#_11OqMWXzxxxeL z3xf}CW({xwPYuZ!S`24)6mf`9ZJXEIf3}c>a8I5vQn!Qs#L)2hJWqQ$YQ9Wqm|;Qlx~PgYf2 z#g3D0&cCYWf1KO7VWvt3*p1<5%H+LCG4PL`FZB+U$B4a3AD@=H7rxe=)yhC<)9T+z z4XpFfteZM9a^oMhO_#}{?q5c@#3x@xoH19MD(HbyPX+T0C6u5IY~4v>?ebpuDkV;6 zWY}ymp}=iwo}%h%J7C)r-F%KPFTlrZS}lJ))i7-{)g4y@pZ!lx9Xf9F$Nljd%y`4A z<;xx}zI!zT*%Gh#@8icLYhV{ENI1Du1p~I@7u-m`QVYq7svV&a=S$nsvhoCL=u}>9 zm@h&h0KD}+KRd1I&b)nGG-~sVt|eL{N$uIjo=}4>gzMC<`MO;7m)-U>iB!Sl45Hy6 z4K*2gKCh*(9xplf7}*V!DK1Ty+%TTDRVyXo^p@e5MJ?XG-vG<`Kp?|i^-mF}OhieA zyEjagOydJO9Yef^zdI}&Ib!t&Q8Z!aTQ4om z(O`E=ielkG_Wqz~8kOPtcr)KB>@}=^B4K`3=H9)jY*Oa@EP`@IO=kF0^V-z;X4`jT z(&i0QPSIjIo2Q!-9Ir&2ii5V@zUtham$Eobh8EtyBzRm(s+6vGwd<44aMlOz9@q6> z1P#vpkFc4j6UDBEf{#L^B;jXA=A2xYM;zYOHYbF$hWN1waZ6Y;z(Ci2Aa+imo>%Yp(5=#o*6o=?+=5^)ipvu~g> z+i1mEb&zY@vlS@BGdb@nRcY)!1-uik|w+Z3ykC#}*W z$=r7wyvFJx?LS@LT>Xg0dcVx{d*y>8{h}zFp-)@wJF_0Wh(W&Y&72Deht0~pzbR0( z!oM;QmD!OfDSTgnswX7}|4C1vqKvru6_a2S z^jX{Vgc0{F+;9(%9dQ_VD)J)fhKV90P6)!kx8{w-y9&2=^B2LJ(M=P|()KV-PESq` zK>`0sb)9ZAMdr;0^`V1HG3d^?JS+DEV1y_YM{_1q(~^uA1!6p98>EMcYfMxABMF>_ zytNRtiN53XsP7>8+aA}JtCAtce*2%kF|jM?K{%>EeWFKqiypCY-=SB4+Y zv%ghr=7Bq#R7H_@LZrZPu^{Eqt`}<1X}F46%)wh)?Qt=leIZH=(zHLyioOH76@dD zLHscDOrc(J#We>-;Aw}{<86Z)qGhUOW=nZ%!q3Um(#Dh8v9{hMe#(1+Ov12X^EJnJ z9O3CTN0HXvDA>x(Jb}I8F`ugNUd2Z%#(J#znsl|FI&ieX6;1MHoqdpC#!fUQ(z zuECE?bvX{bc2pN8ZW~pcSbc(cwBzi_f=*AY*hO-aGJirGWe2Zxl`2F4`=;Vh*Uh&% z>Z0W_Qw|E>yiq0~88N|p3>em6C*|c1DD3DXUidW+0v>C_RZSfG6d~{BF;zD1GEtqzTZuu+c zg}2(y=8u<6K#sH#h`kCyj?;6oIzUdX}pH6X!w z?skU5c}AQwBK!P)c<1#Ak#Ejd&Fb+-M2c7Mf6N?P2~#Vy8<1DSllJ-UTq{%^f3kTP z>Uj`OC@~Coc?*fjBG`$RlfMW-D!-!nbc>yEI-Hl#;)Lp>;n1C~z3zGwMBpo<`8Yd_ z*tlfH3c5V}oJ^mAuhYu+@29L~nTFYFU;HzSC;@THe-TC;XYSlQQwS^OxfZQzU-=o^ z;*J0AO~U_7{n_I;C6s$OobI~%4HQ-$Gj1r0nJI(z&Zk&?HMfVLMkYap*gU0cB*~E< zgaX>GeXJ%^oz=su>mCl$GJ6=U-+y2@x-tRF62}-LV;xrr$mglB&Bv(sLmx%669@--5dlY}Tjb)6)k# zg7|IoE(aEW?S5~+%4(FqpkokP>@^BV;_2^Ot~eE?7MiR>nJ%@ z=u+iS0OVPF)X?@_>_ImxZI5R6b4vxz!^1yVtRhN^!#KRq#7~cUQ)kdsz!JCta->E55_w}ilxW) zxWuZC%?E2%~v<~)volK8ch{op}sp;TP9{EEJh*ZJr$ERkgHG- zUM@{VpnIRqOqHo0%r=xf6*Gl{FCr4>plO>p%D&fa%l& zNqEYf$3i33wy|LR3ERFd6eFi*6naRW)vN7=WjvB0`*>^A6rB}nIAv<2TzcKpK4Lf! z9N6^5j81!k!B9<3NF{~ne5Ehdp3E=ee@n|i4(?d@b)LwAX)j5n6^@q^nZQ9LRCfb& zI5tq4v+A<(=?t*( zhG-V6#|SmojYx^<2jBX=QY89(f8Q_HL-zFJ%A+)LNzSZEEqpVc$nPPKmx*melY*n#(xUcUqD&x~DLMyuY_s%o zjxtR-_o$M`9~kbCI|*MHt{;karSNR#%Tw>G&2afM0{{At5%^wqqpXLQPcU~TsZ&nj zh?NbZt~2Hqg^%`&oh;}ziHeY#M|C3Rpc6&B+kBQr#vg@3QgPj|@f#}$=hY^VJAiYC zkOwO4zob+rjfWzSl}$e3X@)f5`VcD-vTLLz&v};D);tMS`+qL}e79kzkmpQ+-5NTG zCf*`Wnd^nkoR}#u!}JrX)nsP))aBQ-U(XRtRXQ+d2_HXZ9m7F{%42$$P_yI%!3s02 z&7^qS_Y$`Iy$n)HkF$9!>y~A+5JSdbBj6mpo5wcW{@_~$0kYGu^Zcwbc^^nu?Bm%x z$>;njN7Q-pg;eZLatyE_9(M0wq)JYMy(pGx146aX5JYkyB8D<7E?IfQHyW#~?*#qI zc_fs#LxJK+Az&-*(K2V{*KV6lm0~3-TNX)r`V>zPln%WO4G}g_SX^4%M#=rqG2Ggs z_E?#Zq&4B`!v5^Gt)`B98yCKoG>qCho45=@5E*Jfxx^{r+UM2NjLLAQW=C-#!2HJ? zD>yEJNUgur-4|jfuZB=@t=vmUH0-U5l9UXVT6pR>b1T=n)cyJ`j}dW&i|s1kHEBl`{PE9EVcNmGvMLBIpY-t1tc zl?@JJIe8p#zyvMj6M0l>2r3#AugvCk2{>Mp zR%b-;(eNpb8tyTW?%BP#4Q*NEaPUaXlqPax^X8cny}DJmOCGBB>>+QTPPxgaWPhk` zRz-+wYR9R2px)=gbngygJ~DR(P+HD6XMqs#_Jz~myh8;Yq)%IOQ(;DaNeW0)e9&;0< zr5}y&&uK8?sELlpU$x#^xqv)MbGL+^9X~h|oyfo(<$_1zcv zrEhD4Z&2sE39=R&x{_|qcl6kvh_?C``+gX zxwbw%7qGDtwB+pzfaJKV=P(1jnoc(p;_Nm(Re9nWK1Zfw@HTm-5M39#C z;1dSehU03xXeGG4&xvf*JyIU?$I*khqv&_Hza|@q3~qn^_sQdDFR>tU3X+HrLv?*! zzIfGJZgw_!o@NE~0Gdgd!w3V3dq#5@)LF;2ZpoI!sM^q~LTuTY%*)7#>yA`COb187 zT|14kp8Gdnr9J(wJ($iR(01Tw*ng5j?QCxb!hJ9OqqHg5y%((BS0sL#_Ug&k!0Owg zx%v*#p03Hc)5y-*gpDS(0bQ3t4G;@EPKO)tji8EnrsI;z_Xz6OP4*r-P!2uF9>+r~ z!Lsu}_Ghp-=h>pobSmKeV0e+t%%wn+G7CZn%3}tKBpqlYe)ljyT;l$UHl1Nd>M)Xt zmp0-)rGVw*HdR64v}NlF6VsH^Rtjyv8GMwE94Ie)94q&T_H045bKa<-#EX7EvH2%B zV%q`YR$h;SLTFFtW(PdGQsqt?vjdGP0Jw~eX*#sWlidL3PmqUr>6D{&!RlH0`^5QZ zPd}~&VAnsfOdE9Xb1Z;xg*)Gx(H@VT1QL^G`^71OHfpH`VD)bEYZTh;41=}Xwqzr8 z&aTjwM+P6ri38>2qhEV3(H6*C5Fn7t8h06J3*B2`3l=t6w-c4K4 z8JX??=)$)qaMP66Ev>J;8DRBSQy6Ije=dP#OSTq%`vgDj>3Lybx>b!O9f{l`29< zOJK_n)b>IesaY$~dejUDZ+epfrH4ai4X?XyquKa8+)g$XIo3tW$u#i8T|$?8-h6D3 zndb8%WbPNWgX-V=dRDZgQH%d45zvJK&Wv8o|ARJZHgX&XB7csL^9SPuwGV8~=Eo^{ z%T0_u^k*Lm=`aB44L@Fw+b7#oG(06%a#t(d%_Vfu!h%!9wa_N>O8JR+KTvWptAEvI z(x+{cFK*H|n}sY`Ss~2&yAF6IRd%=U{6QNZORAg}>}xUn3CN#>F4+T!c`EY4hVP0s zqIrPFNc{kx+uZDY=WvKf;W)*wrD!>9R{D%8@ab-W1=J>BIt$Z5fV0*@xoiOS+XVZ4)1>~@l*z^ z+IzK#k+#1%rw0U~aCy6%{C8x_!}s0=2c10e=?V+cg@pian}V~9xRHIeA|=)*yOxDS zCrJLHcfO4LabZDtS;up}(zIyQLpfZ;U?`)os(NXHKjG@^cM{Lf?HaF2*U9Dfy4@jS zU|Ix4#T%5k%#+{xi{?L@gM;+=8&D*qb_)jDA+RMDFv@#MzZ==tyk&S~UrhVR zfw<*{=#oz+zTPei?n5=zhqjtaE}a0KcS#tP`CO#!F4bl2R4Du9aza}N)W3(~g1xZm zJ!~)2xSmbXwC6PD_1QDFr4#tonh;4nBX7B_Zzyh|$SPf{40hf>Gc-C#IJdg&{9PjU ztKVcyu$;?)OxP^y0?N^uB}9&VS9Wf202}k9%u6d6Xz#oHNb&&GQ?eNVex_Op*Q)(5;Wayb8fRG3}52es2-m7p+l!eHQD zMUW|q-};)Mq|=;*F((eBcaF=GSBD>o$SD6nB6bKk9wGP>CpwL7mRErdXg6zh`>tne zyFPNWu zUO86ELu%i@6U8DE!RacCa3u2BgNj3Pk5g!MO~b@yzI%T-K~4PgW(0E0DtYWm!^nl* zmT5^cF|L!VYa$DkRUo~L!tLh*=h|qf+Q*sM(nYmE*t~BZ#wv>xq-{2F=Sg8zWw`#% zGx9Od!eCCbyHp{IZL2e`Wrm%7C_9fCkRc5Uhw_EbYmX2h6iq3Xg|a;{ASjuXE18-)VJtP6&Tj)*nNE z{?Y0UtTyYFHwV9;ryT^T309zTqD!&H)NYEKcc`5XJ+|nCn8y1|QvmmSP6U<0JaP}k zONg2-TArEicCNX=KpMR9nB=pWa=mkc_{_tZ<0dt$ZIXYh9~M+ac%cxZfg;Co zsc+?qs*L!zL%y7(gYs)5p-VAGw|!VSPhdN-aZn~{#Q;$uKHm?UP3(Arn4HOjV^FSD zEto)Is0yX;-h+<~lUR6JE}qx5j}N0jxq`x8HV);t;T~AStuz~aZ(J>XleCY7FOM$r|{VJX~930vLt*Q0DxiKCzV*LM@-N#!GS- z4H!M(ufk((Bl;J67co`Qu6<#Al1t;+#>$tHf}@+~aA(J3+ONH2#_-OknS!ctl3EmZ zxS_rM1Lx*8TqFlSY4s^lp)0+$I)Fu{1`{EQQWO5EH8xcd;3YkF8S=xC|AJubBM`t@-963f z`xjnM&-iTG)XDw?h?jqY$H|werOUWgP^MX)um`!+jDlVX<|RX+49CBu>}m)PUZZe& z7blVvy#CQTSr_F98g1G-U6CIqinI)2N2_`rG*DP)7N7?gce^!ye;2@?Ih};)8Q4&05f2% z{=uF$5Sv4<<|btQxbn*%*Vuswi;D&^pa07&eomk!y^eUwxivQ49mw>WbnNIu1ehaX z(%dZBv!yLF3DLC_H_|Op!t(PYrt=|$ zSz+j_*#Q)eiwotktcEcMP5m$kf62@{bI?X@Uq*P&4idKcgrP4VH+H)A8h2>l8NG$R zYw>}yB^%ONed|Y<@r8Qo>n&!DpR4AUpsH>GMD+Uu524!}Y9` zJB=@n5Wid1jMIi^w^mIp29R17vruHqURkpxPtdf--Q$B&4Hx;-x@Go_?^MO2U4zd?$s~-l%s4G!;V90F>`u}Wa>5N{z-7X*s+T6ri73`ZIaw(AO zJ53fdRDBSPxZN)|d`HzskQ({D+uCCA{mO61!vXS>NX5h79rnnr*w+j&liZFN-@dwV zf~Vx-iU69wuh`k{R5_X2^~UdsK)qOuG)GdaWAWF!heT#B`{rWgGMYTPSd@vA5n`V5 zHr=gU?v<5YwFwJZI~1hBi@ctOVQ0$8(X*ZkaAN$`mPLN}lCi!5=noS$^_VCdInd=! zF8{DOX}rwZo7AWn{J4@+@1)Ie8KRdR^n%6kb&xb%!`&Eh#ixURs9Ne}pnR{)R1JSz z`BnLjQI*exoAv&8*f13UDRbM->!9s4HRpby^zq9+AFqjoaDNgr-&3fX%YJ_Q zCiiDk_hLxt#gmgc6)rMKNlBa^hr{)yW%Ux878(y$ys-+;e3s|Y))rp*;za0pnqDDk z#H5dbu%l;ocUtB+;Q%+NVH=kbIt=aNJB>&|&5KtzbIPkr`hHNRo&T_T86>YHUSh&o~?pS=;2qH#BmDgGkOR~@yLco zZ`$hu3cu85{boUG6!2&O`)DK_l+3ss8sidhp-(_x0JSKSUO2-%K|-KGkh@S*-8?*m zoauvoDkE$UR*RNr3zOVqM6osIT)A4-SxHjxWgt%b9`M9Dt{N*RcmZ)ue}C^&QCICe zwduwVKHQVA$TW3y+uu+0+&Tt~%^mi{l3#4hkCVVP64hTS_+9#b<#XCON6^-Dy|1q} z)B!XL-R8u3;rRCOb{DFrexhm-wcX~(mNJ|!_LHDcmhLt`^$4khWVgPT7AObG5ysUA zJ7mQ;2#1Vf9(EdfSm>AHyM=t8dZD%Bcjg&-=2T9KYe|;(gLqQPXb#eXy*%gDt4i_b zj6q0%MlJTVtQ@oS#bvv87Bs+nwzo)izwA;rrsp{Es*62R)XPJ5aY{qfeSt8sCb!Zl z;IX1BFQ>LW1~xO17b2A2D1cw#*;%^+J&4jI9xjKzn1g&*%yaqFD5)oK;>VT01o$t! zZ=yhy)TSQC_#;XZY6Q#&hmma`rYlTyAWTER!?D*sm_kS8GG z5sB5T2!H;RSnkEFL~ud+jpXJ(H^Rf%`-7mh%W4o-$dtEyS~k1Kd~Wp&48wT<0#zkP z%fVbu+#a~*Etir@fp1mF3tquL{r4ZbI&GZmomWo zn|{=x&b_3f)b(&eG} zWv5q%;oIfs3^-7wZX1-d;&8b@sZob>7?e0=xOze3mKtR2sw4DlpMVPF+y4BqJ(?m@ zBW0CidYBG!5;)aJl13k`;y~dUl$4?+BEeBgAQaXiIJE`I+NBzZe1YWFX_*$@zcqd|EE#517cD8t zhaZ1Hb!j;_4o)FGFX?+i>t3JDh06WZl-+g!h&gNS+YmZpu0i#w;?c}MT67veGRb+I z+zXswa3!aqEO;3{_iPzqz@sD%GBy0XuM-AD>v`WG_zq?e)tu(Ox`GdT_!-1Ly6-8= ztEmtt#kfLQ=gKuf+$Q3(Hb5oy)D_!X?c{a6s_Q$s8 zVmYW5)R-fPR?8k>WU2}?bOhqjpw=8{lJJhylDM9BL1$}RA1zy zOErO&I^xIZIIg>3_LCUCXgb*jO)%X~N&hPCv;l(;H8kV7VITafApVd)z@r#H9f?bH zHa||?{MC+VISaEt5nx*JWF9N95@6|xkp1tMd zTR`>;OvSXGE2OHSCRp7vNv)bT@Ugl;BGR14mZE4c&jF@;8E@)S|AEf0f4l-&;{GeA z+UTrqMzs-Uf~*f|ubxUpUDr%SfzI>u^VX;HOy4UiSG!WgPtoRT z@=Y*XC3+v}du0Dzz5l}ppLPylBW#SP>Dh=tpoz>EpV7w50VL*PqEaNT(b;w#}Kp_=T1f;TT5D z_vrCA(0YNh9=H9%N~i2lwsWV^3eA=Ws@YaLNv-C5kfUQ*1wr(NGD*CkOj{MeljY!1 z#7GNzkN>3uj?jHHdQ09rFf6peijM6Qq$-L@(xqzJd&g=5S+sIC3ksl3^jEuqw5WV` zrcKlXz)Let32#q5OM5(-8xTD;QjnKUX;PP3_}ju_tO{sZJCV3xN{Z*vO z20DJhl8Sm*WV1SL;O<(0>0umgd_TLQpICVJ0$}Ub1L1Tm%vYP4y;P59mdZ-UF9=eN;|;S53O`}>p=m<?D-6=fcd#SwCQ$0 zlB!9OmK*5!1r@3?yohXMqrJCwE0`Xj+q+1cZU0mbX@mAT8VLJLVy`!C#0Oaz8C|lN=ooeFPgK0PdWUJ_6>9?I zLC8$diB8zL0P6cO;gJ$*bU*`|K=XwlwNOYWW7*$*?u20kMC}HrS=8K~k9O%4v z$C4U6nkBWD)8Xn3s!%waEzt=cb@jmXeBXR}r>xU}?)!Es(Z=j^5TJ`~V)r>Z0(pa~ z40nz#(eVo$`QZHchGRvR&g#^B;*239?LME+8PvU#`pfC;odNdlQR#Q2Hw?(qi=?GdT=HGM# zo!IW*bi}{uh~E_E-*g1cqwT-x2s&Zszv+no7t#?vl#c$clb3__wB8pBTa@NpXf~?~ z$aH74_$ryr38(lyRScfkp~GTj9Aws7wdvkx#gR<`FPFn#u8ErLF0U-&98o4_KmWFW z%Q*(7o!$%^4=^zP`KOx=leH1TW+-p-fByZ`(BFSbs(Z6eNe*lRRg&N5f4^zaY-_d; z$;?iUa|hjf7?|voz(RIPF)jz!St*0MQpPO9wBoOze&33EwY+|p(FQ_&pMwb3FX;Sy z55q=jMv3f}z28y&b?mRJSyuo=S=(Qat#j9)$rb>SiR~rvqemkE zI2XXp>&S(D>i~UV4z$^olRtWIWm{)APF1iQC(KGpbe-MaYXat?+huB<&#yyDMe24t z4^FPb69y&>l}?!LFYf&;Jik$N&X!82zFtAKJ?r3?>C3>=nKi2Bz`Nd3+ElSTDY~@I zJq$hqyMYj}GY+jMa__?50sPz`BbA&C zJKNSF=e>L&2k`vDFqQ{@n}^?t;G=r9oEfKzI`f)yo&8G1sE~^p53NH3hKLA!Hs;Eu>_&U4kZv(p-eu*BGT~CMescgyjZ&{B{y|Q4X0Z&eb^+2K$APTjt zFSQOxrF*Fa*z4ZyyUt35$G}R3WgGI=({o8>pm&n4wM9nj2;DcpIW?`{dG(9+C_PPu z==8l;>oGTgO74K7y|1^dgIn)59|D0;f-#gI_%iAcy^cgN_)t}ybISxy={+hDlHV4M0SQ~j3-T7o6baR4V*Rf|hg|X3F>hC9E!RSTjHDtyXb+UT=)tp1FbDQS;rY?a!A~ zI$nkKlDl71jfMy{BD_vU*-?mjd|9Ur@TUM7n z60EM)PIoCXQiOB%Wq~ijT+VXJuqcv)9Unv=mG}XRaG^QdCSu_yv(kdp{3EF5{KQe* zRGc>@tTV9Ve-fC|y_CB$GKF;&UZxFTiey+W60w#bfJMae_Y@rtEG%q{vjSBT#M1DoD-Y; zI{Sb64c8`}P05eVmf}2xsg6h3^*Q!uW&h1^`nW0@^_9;grE^NC#5XMJta(pc`OhDO zHl7G;o?BLQNw_|~mhM@8i7=N$>C)z|z8M%Nbz5t)K=4%1mq@d0Q8E`IP=QBAzU-a- z$Z`ySRojrXB{T1E$b9vx(1Q!Z7bi}8dt7o!sE|q(%nGG!kf`DZV_M7sRsJ`E_!Hg< zv6hVIE->=^IUzSkEI5Tr(f)I0tIaJV?w4okjZBH6Cd1zryJ(DWvbk#~(LFmW^;Mk$ zcB7o4g!!Il2~6_u3V%5ozFh|)p{=$p9i4#?KP8KpBVJF^k3qYUht>kdAu;GLAHQJ? zaHXeSNn$vNqygk{iJNLHRk=48h#wX=WB5f?`o{>>5^c*xR{H|GE=rOOQtYQvLR>ml zR-qZhW;3qVOhNOB>wR_n1U_KrW}D)6tYd%=%rR{=*+3Y_DCF^9%8DZ6YoCrPebIuY zNXMK*=;W3v=ot3r)z5kyOyiU|m>0k2PLpcMXaei>ie%B8Vex(E=&%Gz+x*AXfR(;c zBXSe5hR8!%O>I1G8c#Z${*7PP=8$3VjTgSl0^aGv<_1eT2yKMPb|nf|NWY=U&Nh5L zAPx2Rj+{-V;up@LE@y*tele~dgNA7h&W14L$Sahs4e;li;}%meh0=xa3d_^MRu(F$ z3fwP5Y-=kVa#lTKwRZ;>=Rl>K!>L|6{MZ%GbxgK*`MGk0k6eHLH*JZNYp@q?A8s0c zekce{&igVN-mThq$S_Usk4kxWw!_9OeV^u=(c@v>*F`RM+kN|Lh~GehNN*Ub{qBlk9PX$S8n!}&?Qzu{~`y`HiK*y?(mnyJHf#{@Qbg>VL$OL;D7;mdT zxv6lXPi=a7)KtSZbNS{+k*g620iK^UyLwWJSC9MkSRn4ayO;Jf&@|M^IJW{ln7mU{ zZ7&ZgWWGaSFVg&t-Ku{4{T?pFd5h>CNm<=n8QWzRL(Yj(l98=t6~tQb(=%6XN8Ufr zl71&TAhjaV{i0QEqP~AvNM`7!}r)-(IJ|UtQ_45$|_Bv*v0dg#WQaO7kivkv+C+-+M;{Z z3qFN^RtG-Si-~#h^aUhi<=LEu`N-%= z>i^i3f4ppN5KlB$mh`v&o+x z*5KK(0p$0%`O}#)f>E;XYU8BR6{x>E+S%mNkYz1)t+PG$SC@+F#{ZZd2!KUzou)K5 z4g0$I>s20{x&%QsrZP?+{xrQGOW9=H1UY|2#!NhyzbUw^P{V8kVY(N)Ia7@JiiGw% zpS+}clfbJAds>D|G;wttHpy^nyOSm8$zInwzhjxIE-T@}Vlg2LQ}+-S)Juljvy7Mqb7Q&glFu1TYx?Ijt?S;j~!q?qkD5V~p> zAeE|Br!z@zem@(*#!5nrVu_bDmk`Uz?vuo~uk4*N#3F@b;htG)gU1fj!ATpHT!q+|02O$39Bfx*2=hgtay+^KF2`Jr9)9y zp2&pBfs4#bSx2*%q|vxLR#tIs%5dvD{{53TQCoMv;z(M9^_z6$&*{?Yydn~1ia-U(?4iPBpi;-O@ zc91_Rzc>J=_)=aoEFw%RR(oa-tA9NcQb()?5zZ!`q(DeJ#S31i;K)>`5X3rchYq1m zJdLHO2BQySFW8_x0~MHu%A6OAUA@m|)lb_ay>d$>3bABWE%E%X$=A*9o>Oza^TJFt zEO5!xpi?=DVBi|VI@RHYn4T14`1r!vjdf#3jRF4puuF$ov1lgY5#F7%JRM`G)AyHYS^r zf{NPNmbZd~leo?JDc@C2ww=v$)gFCWrKHr)=8S8ac_ljgK*>2$ zc+$*ejkoQ+d@ZIsvP-UNR}eOrc(ST)Ml|@YK62Dt)XXKs{^BtHq(XpbE-dgccZd0! zPGDr8>Ha3_9Fw{!zTwAJ%Q|olQ?nH=WwA`mLN3>tl4_|p_FiGg%838TCH1&II59Bn zvb=op{LTL#mjAzYNliJPPA!fR`l&LIK(@KU-{bx*s&6i!0+%DHxR!ov)Rdq)t-Abl zLReVO3+CR1(2I9QDv5o1A9IMO6GDatM_=Pvd~KjkNTO_cQ~nTPXx}iSy_t6cv7~^+zzU5)Ok-l{HveGF5B$b&M~G5R zgI~<{fN{xeaF)y_zYY{?=LbJzsPR1UQK2D1zHeZWWeS5ZT_+7Uczxrs0S@G+dj=$K zwC4yC(A7Atz=LhxaC)?{ai(W45PFv>)!NB>D#N#Rl?~KYd2iuzN(Bpx z8jC^}Zn&eY6}d(FUoVI_2KD$6rRN?!xgnc^6_v#4$QXwlmMPTKP>{^2V z9?ez8y$xOq2R~OC5p8g;es?x(IVt1VJ{~=Dwe|><^ncJVDexiX2JLybPU`pG<&8sa zkD=@nrB#Lw7_qpFtXoS4ZsIppYw{m2qrM#Kz5T{W8IfWVX7w4Qey`5Fk=6*Cd{`QP zsXxED6gmkxAZLg>J+@J@0OmT;OtEnw_!uB=2j)J@MJOZJ))rexD@3Le)-&r6qMzM|oIhW4qGLs$> z{1*@7xCO#hzqZB*8> z^Qqe`wPU*z@6^9gAPR+zni^lCq+Jn9f_NWN1lgyTw%5iJBK{cqwisNcE?%R#E#PF; z%VqvCbkqyAu0D%-mO=UP_t#2HtP5ZBLaUc3+eFkP%UQ$8b_eyiUIth?=FV zkU>REzVjU(mzF}MxDY`Vs}sX_!(90-UZe)ko112yGbZN8>>?f6QDr!M3XYe0HQEPa z>r_)>SA@_t5o2N>m2!ri<(cv-+vt1g?k%gXi^KVAE33~!4)?$GOy1-b&^I-$(Nye4 zN-PQZsO&uB8tE>PtuRq`TT)L#J${$->zogfUaiq#R?4iocs;Rw$|nld^cN~MF>iY; zQC`U{e&8KFcQm?2#Ceu;r8}xDgp{oS6A|<( zRfzwI=wi*(oSj&u?d>R09`WmD4?j4=NE9}@iQQ7nN-Y`5AE8A zuo3g`lPWlE7JuwZAbLfPET7{ehi&hq?}qe+edY-WN{oVt?r3A%Vd#<8j~&j2_p&S} z6Xs@4gNkaHUZU33Nok!%RPjYa83U8>n#ugg(v_yv#(kGF8}{qYcBBfdR|4!fTn@=b z^I7TM|1I79H_z_XY<$OC^)BD=c1SM2D>GrQg?Y9U)TM891Q?D%RCe^xEiA9m1Ybhc z^tM!wm%{nk&KmBsLQ(bKcIAJ|6YbNx){$zgavt9PF;p^j_BBdUk#C*cE#GBOj043z zq=G^-%h~k`UMxOc@&?m9!9nHS{QGrZN-9{QPB>lPberd8i-UMs*Jhqh%!RC|RI&TV z`6ql77oL7ez-4N>gf56%w(FG$r=`k!{#va1#hH{V?OJuBCaYM0XP->GclwvnUuY%r z)VqMwPBNR2X?()7I>(yR?fmGA&hoM%i2d+8o?|6RJk=Gx_<+f{8PcU)6?@kFf@g3~ z13l>5lb!F4s9h6-Q57llyC&QenaZ@J;6?`Mk9qFypXh5EPAUh zk!?5g$WCD8$zi=(EmHHYCq2#QPi6)0@fwZ$tSn|#yH4h!rE#lgb87iNt1l3+Ol`ta+j(kvpc3oH+XjC<@egfj+aD@n7%MkZw(MmPb`OphcpKIiN!~srn?~H7JUV3;gljbwvB#`QdW|2T0t{)HRqXEU#aa{@rSOq zXCaC*NWF=?lZ4v0k_A2n*)O-HetA~)RWeoQ9cKQLYP?$P=@aX0d#)U`1N2!u_4`GZ zez62TNxOmJMa}r(>Kkekou{MVmI31g{_b)zL4nxQfhG+M%)p6+fmt|yBB=z4AWzKj zQ&x4!1ZA7E%~@TTKqLa1HIl*;EB31Gn2U^Wob>gGh?DEVWP}9u+Lg6yEPreN|EEo= zdP|@}sR6eB~sykZZ9SVPMCIer7Tw~w7=(9fR_@ki41X~SCvS! zT4>^EpCpA?-{*&*giwgwd+m=dw>}aIs>fdEcP{RBxqK%SCJHCcm#MsvLHTuu97yoocnRh;%7FI-8XJ9(|55{e<8gexy6-Ic0;lZ@${w zFHusNvDgBIK^zbFs&C&fw+_M z3}K=xXtr-1d>{_T8%{Wa5`v*uEGYb(xf-=^_^jVDI%_x9RIiTHUUqFQWZCb}UtyMv z#}5CL1Y!r7Y(~CA&d9x;E;VgLve>VUXggeBfmEJ+e|aG-Vw^2TA#-+ZxM!BUJOe?j zIZlkt_j!0$L*Ln)N;gora)+#4)&k{-)DBS8lpoP{QtHZ=yBH0-Z;%oyR`x8@cTA01 zP!uJL1?8>11%)!D7e)8ilGGdO&^h`q4!1x3BR}~?-V!^Aiw7jME)E=5mNma<_;oE_ z&Oz8zpz7+})<3AlxIc?*Uh>HYOrUo+V+>^x3&cLoTC&u2PE4^&m2|3fjDkCPY>gl% zk0?YRuJzEWcJ=zBePN}1g-uCwv@!?(sHYu~54nGfCsAtT2AcC$+$|e<;fpXA#6vr_ zo3rX9y&^7^(&MvoluUu1C2WzPkvhM$YfU`rO^`*dbx#4V;+lH0Q0T~OtSfT&JE)!Y zezv;&A{SDV?oq?>#p8W_q?tl=31L=&l!DJ=&)hLBtLnu{A`DQzV$no=hwA`wGO7e2 zSkluY{-SRt0Y6R_nu^F@E&zqL^CHR_S&n+jVF?L=zpDdZK zWs!7=2G)tQmfE!*S1~?7UPgnvzNP(VZq|5U5>pnQ^+F1v$^zwN1i{V-)6f z#83hkBPzo_<>5oMd5SVKc#@{W@mt=;Xy=aV58apa%XisbJIR4GNh`LMn%`Pt#a9fs z8a@)%)&v(Av<{efsPkEW14Vu(hbHfB_4JVF?7ZU2nl6rEfAuW}8xX6?3WCbBLK(5@ zSW^M7n#m&rOBv#0nmQ`tno><5HPR&WM(9OZIetLF-&6I!>58{j18_sPueE#9v<^`Om1;4Ch-h|yqX+dg#-E{%=^sz~+ zZ&t9TEqB)|WZnXsRU}WF*$I)gnLh}mvW<@&#Da%9tpAkDZtchFV@NJGAdi26MxYyb%SE@R2Ml}{zh4cKQw>M25S=h2FP}p zC`fvNx=aSvDH?*E`b^}h-H27!yKII2hQT?n1ZgN#NhOcLN0l1z$>1R1pOQuLA`IqZJ1W_~b)j z3n>c9O&N0$kylb8BDAj@Y)#Cqj8RY|KgOtGt1ESrq-ZJL$G{O4mR^@7jJzW(jl=rp z@WBUZNwjx&3}3t|%Lu+vtRq_brGPewim>dp8cF+(CJEkSX1KxE;znFlr)jljFcj>H zL=49J4)Ai;+(bEaX;x0~w?KFU2V}KM>TZyos~mh zYa@8nSX)l$Dv2F_@I3EIHj-*kVZ(4H{()|b;Hr8Ac^u9Y;-xoU_AfBL-y@XhsxWLj zw^qCXs=JNO0}=}Xk>GtWejruaB2B|A8;99`IJ?^yL-0K;x$Ex1gY&)d#=crYmEuxH zG@sNf4w6DG`o-@A>mQb3->?mC2ce;=8WVICv=S7hVrb8-npMYC-ulT@(0u0%i70=;(bFU#Xuo>)tZ3C;Lk#9vz@>D z)-^wcwiO!`fhyXXo=Q#@IWv^G@AW7!{ef$$KcrPN;ECERrd0IMTpoEt?CPx6Jdd%fj3XOgXCuP^W6V4z%a*ab& z83WoUZ^&NUd@!?wRZFn*KCrETrf}dX#R=I5ScAYKQ+nu+4|e`TaRhL?0Xa*fHmB1R-Gauc#iwUe2LBU0}1LzO{pA>hhkY25M`psxDG_Me>z^dCXSA z{!7}Y*LQ`2`iCO#x!f!Ef83*e^_eE7#N7XE_15vt5#wgpDX=kXk6zJ!covQC9Iz9=%UsZxIhWUwH@Xp=&o0l!Du2e| zz$lZ``<*=Ao#obr!3TBzkFE9?z0=*(#3(B#5=S1MM357m=bu@jb!RQ;3*L=2H78xJ zY==)#5Oa&ni$7ck0`P@cEl`q#SYCDQt56m<5-ueA-%1tq*1y^3hab{OUW>}UfQ|P2 z(FY8BTHKfK=1sqOv!N?}eaS&^*Y8d#Wg+T{KGD>T{8pP%nlXO{yE{pkajnF5=yUHI za?l%5APZEdA+$KS>GyAn(mZ+2a@+Ge`D;2sN!mMM-)W^JA4mB~5@Ldg;=@1DGDkWX z;VFi0iQJ-NjKq_qaKLu|;Pjv*QvOjue#W!h+ZOM`M5iBN!T>Oeg4d}`ljp&8_CCqh8v%6CWaClW~o0^4d(vl+#FD4 zs52#z^nufkWCz974u?7%?+4SVl^3pVnA)QAsrkh{B;jsH%WX3Zk+%Y0qg&}h=!^)n z?*!pwz1w~#@{aR|#wKke-JOpwzish=bsKPaNxen7>j^Uyg$Fk-xm!N}BskT;&?K53LpH(z&Wmsc2vMD`#l$xihY?%+{4oeVAmSD}8P#k=+npT^K*C|kJPL*-z%-$CSlKlW2WuSn<1|U7C;ywXCc-hoFTWUzJyiX zk_QA+)U-yfK(3~^x;WV8?G6{@1|*|?`(=oE`|=aV&%K`!@!0CIfTO(J|P{z8|j z@z=XY0r0bpkB31=$Lo+GMCRZK!Pwx0UbM+ZFU8(gI|Y7mMN4*j+rG7Gi9(DW|yVf$L?9c3_Fq;ZW7TDm33Q0 z!x(%^oir#_Zxw_%CyZNLnB6nT=YSZ@Se#Gy&9KZD4q1gydpqXy&hkUNs@yBwYu@S3^0LNa<*rWtk$y* z$66;^H+($~qOw@Ch%;Av?K(|e!4rPwDoU4R+_zfzRr}_3{mZp5Beva=Cj)p6TpJ%Lv z*pOZJw9ktzmk+O0Gq*Vx!e309OqST}*nny@Yvf$v=Hn63W zGMphiJjEu(Jq00fweRqw=Xhr0$SdTCZ1}UaA+=|s`$1EUTMeSrOVOJLQHe-6-}h$r zra6m8;9gcE1YlDRW{Nc-NU zosQY>%u9(x>$ps~3OGZ6<1Eq_CuxG6Reh9bW7oFQ@GJ!6rj?CGBOI zr0aOUdVH(IC1tqxvM=7v>5lgu-w{9M7ZwJa17qwLl0FYx7(?T#1&g#=i3^Dv3D0rW z@c6{$zr=ho-I(8C9!)iMGcD{{?>p>g>EF8J*uJu)rg2{*v3%ZC!eqB=r(ddXI$CBx zwqJl{IMzr;PM#-)-`XK@OjncMI(3QHxc`%l#%HSr%1HSL@bX40%_YhE&?Uz9Hm}r+ zpbG*eqHOBV+1#?ABIA0-f+r8%XFp1o^Le=L~CF+4owc%udvj&zWGcekD`bnn(I||qYu0y-4x(U z!@-lRNL3b=X7wdyMa_NVkNVy1sS=mk$Bw%rhgn6vbFtRia@uSfK928qgYsiEnekKw zb-7(sAQlswOXIeaFEt(0$FiJ8H8(ZeXO7E%lm|GIz++1giYXdC8Q)qA20hx2f+t&` zv~zaO8PJXyS@A?^9&bRuygX%o#MVm}^o$gJFFMmowUL=}*&#hB-fVQYcX_BOm?mKla87T_Lfbqb1tMjur9}leX3vz(b+ua4L%9nwiRp@ zeC0W`zi=)RBGT?G2|29aIqy3^4#*6sZc7hLq-eTFKxNNgp3z;Ix0JgxY}}dKNkR!1 z(6wt^dv_>0*&Pxcp2)}z72w_(I~pC%(HeZ0%gom&UA1PSzIP{%qS#H_+=^nAl)MfYhDfoCNJ!&a->#pQi1*&O%Qr8>(o$$$pfsC~Rdkgky&5 z@ka`sjg&1gZJTd!Mw^ZgCZ~d(v7B|!3@>P?*fZG$U5PLAj@HiOc-RS$D<^6)C$&3a zYUOH8C2@_zz7?>d-r3cZ+!X9x2dKA?-o^7Pk#ULRPRGBfJfnX|>ssQ0TX2 z2)uuLJ7(YbqFE31=A&?)4V1=tR1{WR6fAzQVrRiXpf{#MvN?kAR9@XVV2jxXW$c^p z`8wOFt0Uf3{^$S%>k^Tp;{*<}0x|6!>%V&nR z)@-khYz>Ur+^p@c*FoWTdj=d@8#}$Gb+fjzaeU?`@Zi5Qo&m?#AG1H8{qGbfOMwUK zvae`GY#oegdDuAEI35VzprxhdcQ7(}rYI`@>vG^Pfd^(zPIk}O*-*<3lX8>iQ9tTv8xKNtDEj;OJtp@X@dlew)8?e)5^4Q!pA1RgxN zzR{nbpZhd+GymV6Y#e_*7Vv=V*Wa)|VdG%`vo>%k|MjQOUYWZYTWN@zTLUx$?jiUD z^n{21zYG39U;Xcv|L0Qm|6R(-@r3LDUiyE&`hQ=l;%Mw3VrvcD)JgDv4EuHQ|NZjU zh5YQ-&;EaC@srU1eF{)o@CHBoAFc`B@G2~70~kqZE-J4KyaQr({ez|t{Cf2B9XLkK za-7+@rHO(fj3OoaT-gnE3xb)XJU-jHXR4sU@=8?s?n{$iuAMPdGQQ zbnZN$8cB8`?P0_EjPv#mmO3phB4zF7iGzuYiwl-ZTqyJi*4OVicsV~E-&Qy1ZKLhw zwL$+xHdKl z0R1R$({dm`U>g1tHBrzsUcG5OWpNwH1^pX8T5qGG!NS!!0{)(r*H=VfVlFMo#7ISD z(o0Edr(zb!$NVEp9qLiYVUFjFZ(#o>q~*0JetQ}h7>eHirgAxW|EZi2mefn_)KS)_ zKL4}~iXY1(LN!pclVZ|$sTT$YB!z-1;{T>}T^nfjdkh@9x9@b3rH%^WQ40MNha(>T z;Sd_EVtFHDU$|x6NI#EbX-Y=Q_D{g9YS28U51bble*{_RpK3UN*gZfXN~(k zkIBJ4PE%hP<&y53Ci!IR94~#hFSi&kH-el0`0I{t>kk|(2SrYS^O5JkrIagZ^CP!c1y3-{Y zdcAqCeW-8X@}!>c?MwcFByN@Fa|@Aj>pAlAg7H{S>3L?c8>Er^?s75Sh~plDMBj)S zsE^L-K=MlQJ^Q2s8pxT@-ghG7y)WERe6BXUZa>oaUu9KpL49ezSg(Iew0X=XP=B&# zIvCH#x#@Vc^p4zNHQukKMOMg*J)D1}o1Kv}q%XDRTM0Dn`K9quLQH(R<{0W<+Ej;J z;jnHpy?kZ!4Wv4)u=8Xwm12p{k(GB3vufc3*Eyd!fsPH%>@HG0O7{oCh~wm}9(3-l za;Wc>ip97OZTx$R^&&j5)rI0S7poYnrgCHE9uXayiUR`5DfSiSPtw(Lm7ET1f>PAG z5Iwo4ix|_*m!{-`2Cpg;#kF^1wQOQgFR^Exd92UnG^=bV?JqMlo*JA!tvlbe+|DkV z<%->Hxf)<+P%GBn7fnS~{PH~+**FxLnMqI1XOqdI&cWyuiQ6YAg(V1nQCzoLZ`9jJ zp6a!)Fb4KCnm8x{b8wl@eWM=1Ud0M%mmzTrbbYzo>42o*wqU@g;Hp6ARl4c+$)nk| zxK>68UCq&r=c`E`xk)Eb(0UxN!%I#8#!8M`tXyswl7Bpe5X#h9s7dCKK4r;0SIqZ7EpjRc@AzAehRpZV9lg|GY^dRZS@X)s|9 zBKYKt=>e?++3q+rObA_FSc9CPsXZjatb(zv>t@Gx!(#7^mx4LB^MyY5&9iE8-KdB?me4Kyhp;W$(r-|Aakuq zdK}b^RDCesb~(Q{F`A>Y9(pq}fRyoN+_(4a$7)i&t*4SueB`YRvf)H53*{0+q2A<% z$~Z`aZ?FNg>hr%f$RiSOLiRRb9&<9Xtf$T!QED$scbTeFF0l_I!Sin1xdZ8bb@ioE zqaNkfv$1JMtAkpL*rwsWuPghKDJaLNDjgmZX>fD+oR&WE;W<&r2*5CY)`C2zF#eII zfQ$?@p%U6{Jh0|D>t$>BR-6&PPr;MFQQ|vep40mclUdee=jm`FnRwVe7MVD<{kR&l zc~y(aV%waP?OEoS$8pDTV)wd3)`U2*ItpHwqvH@mY<$j0FVQ`775 zCt&bBf2+32yLLB1gOk)F`d9!sA6q2?_iw%2!c#m9P)Be#S?Xs;&W<>Cy=fy;HUs(g z?;j0UhSKTlwyXK*o0q*AWDb4Y5!>LEw($9Ae{6zmxBBA7eG4XJ%T;tiN%KVU2`pU8 zXz@!OPohH*MLcFg|9LT=jP%!o0Zxrp*X{s(y{=yN5}d@CnWD8!DfTXM)1><`%qr0o zHnj>92{*?^vt^^M;-8ttkLOQteI2XYDz33cCe|e5+ zdQY!vAD?&e9g%-NaztiyMGOYR&q?t}=+EAkRVz`@MV`V4Y!$h38(wadJDd@VWeNK^ zu1xCrJc%zCidECq*)D;FR$5G}4m{;d{nXU*7A&*M{@J<{>bYyJ(IWUYbvLxuW!4ud zC3wCra<&d5*y25y7(THhZyV&CKZ&i^D{k1qI@^Tf@1Mg9+a5`Kjqe)|j$)p~Ew@;KBhN-&o+Uid3p_u%Clk%Ux_x>A z^`!$0S`}=kci*@~YG^2$C zY20Dr&`ni;m8YmuxBe~IXt&w>m~Z)LQZG`^XPQ`9yb_9k?g}Y=PV|v;dcWmj-_d<7 z8JzXq+O_BMB(4U^>y0?7o?h&YEqX+%=k*cNDZ}26llVlp1wCJ-z&!P{-N;AAZqro8 zAfy2N#8BGbrv&2AYwZ9Fy^4N|iPtlb%(pTsFT~@UQIc7Sd+^tBB`+lzI|GN0clD|r zy0Yu`%>AQHwUcny^GbIHlmmV~$Wgk>{#*s!@^>PC|8Ly6jb*scklQtSm)lOF{!at( zso>ZX4S3urF4N^=AOj=X+zFlRuQ^0|No;2aq;{sWd)^GS&Zj?~GraLq83NU$v;`1+ z8c41`>jnSxK`c$4LJwV`DkG^h&lKzXl3i3|dd9Q71KC_I02an*R?+-WV^!VqAb+;( z&R68T2kP}i{7@YezGO?LDU0?z-YWarJYQ}it0mK);2h1n>A-rhXhYDWrf2;_H7;(l z5f&a8_6}=Dq8zc?;tyKbXx^XDDnYcr!|L$8x^(SE3wK^|g?h`Rtk1SV*%EGpd);eW zPX>~yV{Z`WPkpJrsMSaS6BxM?ur|C(|K@ORo-;G8=iM}{SJ>&X8gDvd-n7w*_xVJP zgU;soB(~_D81X`C{D5zF)QR*UwKqiM#ytY>FJJebUhRZFu%6+G*=<~IQB=qDAsYy90h6)77^?|3iB|cMd-*TWV<<(L-p>KUgDrz zT(NaD1Vy>bfOz9nU+s$Fwrj(+^WC1QA@t5iL;u@wneZ_oDUIHv|soP7#@Z?{jjmk>+uE75uF|Orp2Zz2d@|OXPHn zsp3-K%lCPgWXU{axs3pNvQ~dI7BaIV*qC;-6Ml3XzXKm^R%#WG*ews{s&(7@BE0I7 ztXHPUBqUj787GmYT_^1#P~@r-?X6qUY1wh%up)CVCM2LAahKgEG1=#*vqaZegXr7a z8bbmpXgwENQ9T{M?LbdR_%7`fi)nFe5qTasLq8A^z=aK^q@k7VL@SPMhuhPy(J&&QH47RYq^)bM~5wt+k$2C5rG#ry?LFqhYJ?GE-44(q;3< z7P2;#KtL9dP49dL&7KcqbTo8QX)@Ys+GgjeG(=r_Z_b;+iZaO0A&D4=9faOr{)x#r zAz;UDGMX;35@GnAiJsHF!Cv+T7wJSEf`qLl3A87CuI7w-UbnkKyncmDno9T=JJayj zXMmsBcIw2YiIR@VpRw;h)M*lT=PPCx!3vLp7dH&c?Q)}*iv;i+f|6QZztzW+stNS$ z8doF5W0^&SD;Ma_V`ai`f;n_^xY5s&rbmr4x*LnXWF(Y zY!o^18rewf!{f;QL9ZflFuv(KfuOwPg|c`!wMoTiOO#5iwbcQtt^fh#`(cpl4vSW7 z^Ey73M~2@mCd}6Q&MdzLBF50@mEO1>{7EQ1t7dg%FdK_OjVtF=gI{y~EA)$08qSyV zzE?U-dd*SFC8}Cnue1Y)9F0xI-R$utLMZ8h@Z^T37JA)yTa)_>+h+9^q>}s5xY}}m zjV=*BGSp?Lb(7DgMe9b2Yl^n%!0t*M=@B^ep}rQ_0iHrW(Bdib-p8ii8NN=F2zCI( zDf%)cX4a;mO_lD6eX{qGpIVOOuFad%>(F4@bnV3U18)Djl9gItPyhatZ`Y|pdmvWF zWU-E(CYP;c+=lDm_m8W)`t1guN^+OyvY8e!3CMCD>%^&>yDkCl_V{hy;CIL-GrL9~ z-*qLjL#cau-c<_7OIN+2lgPv;&FjQ!J-G@*7Zk5~e2%it1gRbhA!~t5rI3!>Vr;d;tHnT?Pv4K=---;@>QM6=T((Q6dzxY(ZyebV z>T6y?<5+a*-1n0#O#2hK`>_n+qCnJ;>C?pE(rn^yGxdlt#KQD5X@6reY7tGzrTRpX z7Q8=}y8v4KJmX#(k^zo>wZ+uwok-fhur^Ix<@pj%mFnmq~Jw53eWBfx4f_ zZp_x?;1}vt?aOKj;m?#nK^T`1^L)c1O}St_KSm&icTWM@?~!qhwT}HEYI-3Mll7<4o{c*NXv{-pb${@ z^_(%;DN0T{v08SeVYfMQNLUYCt2gNwXBlCy5$$|Alz!!;klz;dkR5^AaIdQf6_W0c zb$pAQUvqzjkJ%}59qBVyFP8l>ULQR?x`+@2OAoAU^pCfm@4>`-zdStfU(Ml( zoF4Gy^4sf#Uo?R0FMhP0jFhWnjJa()oOnv0!}L6zLo0WO+Sz8ySFp`(kjSQV$;e>i zZ+N5I%gx@1zJ9J1qqFkKZ6`DRMug6ftqS>W=91$5pt4;^{ZRxc3mpurm3JI^pXUwu zID&c~=~T;_TQ>R^8R3Ufh#lsFL?G0e?LRSK6=H52@GKQkJ7`jrm1w_g)@k12zRHdt z@}75sU1eHL2ko!KrFJLUHZwGwY*o)5y#3pvof}Yx$?Bn(iZTh2KU_7=cLnO7_qAhJ zpQB6k;DhIk;^NpGbG+p@+VNk7-X#+QK-VjG(tlTuUHttJ?Nyr$Iovf|gc(Y@Ex;?ub4DsLT-_Qe7& zWyT&%sIsb7e<%302Kqx@vNp?76)lA|1I zJhx+dat+`x%ttE2lyiT)aoQQ{ADAsSuE70)KQ=jlbMS?HyM()QzS-M!_$A`mJ1nxC z*eA#t%aXX)BB?dA0VyJUqm4$oa{NrM1n)uTZ zql39!D*TY$xvU}VzEMT zoOg4+pE;e+#eYlWYCsWTV|S%{=%{k$2TE_BH9tXg;fJJhnP+RO=a%Z^$fxe?&m(NH ztxJ6I#DaB=*UZln!B_g75Js|#qsdedJS(rT2cFWE6t5Cs-=Zu< z9bs>29Kk3VyxqUPjc+s50A?R)Frl4%wy`iViZ6;^<8AS^WS&R&O_5w&$t!8D*$P*p z)>gku<4|5KDiStzCO`rMy6t?rOq6%8#)O%Ggr>pjc#9mg*#5+2&XdWbJ!YizwPW2E zGllLra4P=NQLOuJI%-26W=9Q>#92(-#8|LIIR(mn9%5o%uIS!Pt;ivKlY!6dm^bcL zR`5C77=o01sq3YuHUP5-nm%%zHr`xIftONzj=xu{*Q_lD_v(HaH@$T+Ur)i~#YV+f zwZ{hg|8k*t5tJ zdAu2mG{M!uN;d$WTUYw+!8+uQ;l-%+)|##n6$3rCXbaJq23d+e)h6IX=RsY_a+!qZ z85o1hN?EDe_G-FBe6l!Ps;->)(%e-P9o{6ouZ|8w9C?dNf5PD>*B{?p0lC@HRy1k( z)b1^`p8oJ_mbfNeEZAI8qWZFuTbo!h$#uUwM^Y#{{c1JQc5gCIq2jF>haFd2UQP-_ zv}}Nlfa$YL5_K9)6O(6Gk;s9{`ObrjIPTQ7z`ixjKs{f`L+4G^Dp+0%#T zT|OU2r-SP+ql#rKKbG5MB{^OAVJC7LmCyI;n<>%~V*U1Xb+~5T+i$2WXD0^93^SMP zSGd9Vfnu|CR!`P;{?R5+RJN?vG$5TmJn<8qGkc$OxfZ-&9qijnc7C}`)^VCY3TkvR z#QQT}`3!O&-oy9T&8s$tzrmh5(kD@_T>r4 zwrq1IfbYTzuUMW7UU@ZXJIX%UIB@fA;GbwceO5(r`n8QBM5;3kRl?VlDMXZL-4hTM znYc!l9Lj;I`2tt)8Q`w{E$nydFva{Qorhqu=IR{2b-o#2KScT)ZH$O96(v|TBjV3t z;X;`nq5ItvQs&|SKLK+ux)k3^tD`m!2 zgv*y*Q&k!&u>%!_c&Mzz>AAQ!gq>ebXvs|NCRvu$4e;Mz{FavUxY4YcQ!hb4V6$BP z_Ft-^Lx8ZhR~|@;L7D*aiavJ}YEWFjNKgw%;C_SzI zr|)#4udt8KJ~EoEW>w7WY6mU87k;%iD0D?1dT;W^4BLpPv|sLGHSP@`D!vyaET4&N zxN-5zLxY1W;H+BJk^$jl%eK+KYEwM34OlP_xOe@Iq5I)58`6V7Xij8@od~B7*@w~Q z>YeM~@=_v!%l!TkoV)D606)Za8+o>7du_C;DNnKt+8;ZNt4vO5jONIf$8Tle9tzB~ zp1M261<&72Z~M$KAVX4e-SnvH#U^kUS1r;s=@ikMa~RXPbRG)XA=Oo%3Ja$Zk|tr- z0Q_290Dhu$Mu1=1Hc(kr%2#DxV+(tl0u(GPfUxf1!gYH;_Y<>Pw`M)+&Taet7bl*b zkez1F-h`G!T^Gko7EiEzZ7Tc|r~@v}9NS`2kxwJ5di+Z%pJg6;bFH3zbU0fPbB-zX zB{~PfLu1bduY*qLrg2S`^_Hq1euyV6i^e;xyvAT~Yp;jQiwb<@xy$Ev7oYR1N7rd?mAfY$f=nLG0x`8Up z&bWd^Wb8TIIH13pUYl}H_uI=!cByC4tnFcwVOp51(rPk!fAbz!E4vWT z-U68p+@WYJ;k2A6d}Ki;o3=ZQ)e%m~t6N+TBK$z(!Ra@VzDp#Jm_BmdMe1nQm_NAg z9kA_VgJBKxPHFO_LoCo|{O6MFb-T^wF4`4e!Irve$6M2eIY(mXvwCDg<3he31BY|% zAfwm=y(Ud-^U>AkT1VEi)zZ<~&QVkyd(W;6J6}d!SJ)>^H%8ur43-y+obkXw|G*v{ z<_yV_(S0IM@${it1t#8F50HXyH%j{L*>s!%r8)`f-2?Skya_rj8Wj`!zz01*Vfmp* zpDg)&#Q)KB0+)4*lm`m^J3Fm@=*QheJEqO9yE)4{4bn3!Kr1Pdf1;`2F0)J1nU;G8 zKlh^3^vQ0EVjKUF{uL74h05E#99rQpYwbFj6ag(53-z*)k|CE6#D{s%~?FZN15SPzUk z-O^RNmjM~yI(>sgm;K>}>x)EQBC=kzD`e`f;xVIxzuPcbScR83H zO`gkaxUq;;A`YH5odNYa;O0H(;|wvE^_?wZ;PX|A&e97sL-P*j;g~N9Z;)>=JaB+G zPxeN99rTR;vS!fKo!J7|MMr;OpF~Kr{?~(x6_4WUK8t*V#)xw^*1Zz2y!5F(oaDM6 zn>AH689Z;dfPc@Z@M}le&G?mzR9_?}$hn8IWi!?1yliW}Nec_#XNK~`hYxx*K~CG)y>DDJI`otfB)>$i&es`jO9yw(evZtLgqyQH^d~ z(X7uW&z(B8o%#dntL&uVjDQ{>AGMA@kNPE6a)1NzP#UTgdmU@g;>;pK#`0CvG^@>Q8GrTZfP&a7LFXB5A(SyK zsQ0A4T0dLS^moYOUS}%kYr#W^Mt6q@XGG534+&NxoFbeb2b0QL2^hwmE7^>DV?K!l zBJpN2jk-OvOv9Uz{P-_`1mJ0OD5Itfo&FPT!&tmE~z=fUIgA^z4qF?j@$=LKFArd zIo%V1DU67QX@vicj~tQl16LRB%zd)AOtHpQ2b+Ab8t-QewqSV-CN|9N2>t*&g`ZK} z5bV^2ynxbSc3fwk?)l#%QFhr|E?z!!wtkF~15@Q)om@x8R8^Kg_pSUvJ5 z$vTD*_Cte7$Tkq`UxIJ+p4uz<+sxqPzl(R9oZ~nUg;FE8fNU~;&||@b#Mh+jgU=e5 zjiSUYYJRfvcT#`Z&jUavJcf-A=#36S_FNNg_IOk8+76zCQ3=WT2S}7V$fq1e-F@to zt`n=N-)I7x2pOJFd!8Mh=ulF>znRu*jGqQ+LV!0ePYw6S)hHUr{pTB;;rFwadg<|~ z$3uqw1Nn*W-YlfZRvk?H)-PISFxmSC=}#}8qI4;?7$)yOt;XUDDe*N6F!wd&EdOpa za1450qFtvPaMOE2$EuS%!fmVz*J0j$4VzWzgp@JtfV0Ok77i5gu!^FY)!3qO5$1JA z6SJ4+?|Q!@r!vB2x}&m|-w}lw$xJT%NcEXvAV{Pjom~wNi6-ray3)|wM`-iqlgS|b(wo}Hwmq&rhMXK{*4eTn9)#6P0 zwXiQk;SQJDESg+u9W8dt=9oJ!kv}tpaz%vTNX|ni-@@?gPyPV7S^=5 z?xOt5{t=jh^UzD2KbHe9eC5Kj`mmt5UKNNGSb_2fVfLZ0u&;R16>kM;VhjA@*c=;3 z;69!?QdaMq^H*HA!GOg6^PAJl{qYX`^4VH3PNVOx{`36Ghd~AOlel$TYnMQz{Ibg8 z9Q^a_^fPVMe(x?4ai^M_ZcIe2%09lhmrhk2K1jYbus*^QY+r^+nZi~=Ro#P`w^BSqU8|G zNwyCo{kLtH`~oe0z7~`U5t!Y-3d-9zxFPI@Xs~FUkU?D2L_^WP#UK55b%N+l>lgETz05rqU>j9uZn}Kg&|Avq) zFK=CJb$jBs2d~(-f93kOmgFZB{3w5R{iT*(cANsprnR9#l)%3?wEvkFb3bAIMqXeM z(~Sekm470xA0;zj6~Y9LD4+hDRS@`T6{6_-0fd%7*oTb2^iRK4rAy%&*RGJ0=r8tf z(wYjuv8Q*F97O+3dBg5>MYNtap6yG`H=5p?s-=A2`}L23>y)6CI^H@HIo_J5I@mc# zN|4xJ5s;DV@mB1cEl2%BIN>cc>|>e;h^J3n)74g}yW#nq0fQm>Z$Vig!&rCZ!{X=p z^MoMfH-a@cS4>QQ?m08Pb@-&kE#^Zoy9$F;R22Pv`zA2EEc6HslE=`x2_Q;if%0~a z`SjiYq30vCXqjR$tXd9&5C zgR?$S=-33JO$6t2U$PCve(#W07TBQ-k@$^2h((a-^$uT~F#g(MUF-01OD*vea+s`s zqttRrM8x`le8t{3u^WjC`49}XtoTQm_kZI`^YF|>o}|`2x9?qL%`5S83kA?sI&y+5 zcO6AwdJ48}LH}b_C!5`bhG}=I?*)_ZC9yYOh+vJOYYxVrjj5snv?MnCgSB5c#T{C6 zx!9WK{7L=J2X^n#RYHc3Pu1o(U8Pq|_rYH&|af>_A_P^Ib@w-X*^knWW0Ml|D z^Lz#i;iNz5W&{RSs>qoCaNC?NjlA&q3Q=-#ww9_uDw(P^rd;^pxA)J?L-DpT?~!`_ zd-KBez*b^yD({B;cZzU5-uIcSCDli4&7Zo-Yy%E^$>jAL@dp0rSA+su8P6TukilHe zHlN=H%}*bdZ3NSk{nhpMY~0YL!YW=bFN zWZwDppVms({;xai=Om&3(nmj*$E{-ZEFMd_Jip|1SrS;!VcC2w?U&`C#k&DK2&VkK zUlRlirNV`m>}*NDXI2{6LjKI0Xz0>Th(7=ZAdsINQGV?(LJYH3GEPjMa{QNl^`m4G zPSpzI-=_TCH!yk*+_QW^Dx>G;HIyXR5AmMPu(9Cl{|FR{VPH*vhb%QzPB8Fy%UMJE zO|X;3rjvrkFJ5*nq8-+){2)LH4F5$l4RJtLSse3oAN_p7ZVZEMUwb@Sj`?x@|ItSL z7j48DQ$>D>xgX2JYuY&1=d%2!jomeEPO;y#{TdGOW1#~cC6-r3`R;!f%g+d{3{V72 zLb6>8jz5v15uKkp1u)%Xnh`yi{!8tIWA7|H(20Gfqx?HMai|3(i1z}$b@3NDg=250 z4YKwoJ^nT2Bn5`tDu%n)Uj33AU~tPWl1(O=?Q!rg zBJgShM4&{XF8uh-)m|KctHyG~#6QgrZTvM?ZBTzU<`tuWQoie2&a~U&12Lnu0V;D< z>i;P)jwmCcC0%6e>AJQC-fv4SX$gpGR@I}GKYG5w2+-zyTJg_FP(&=io2)DbJAns{jA8`G2E2yFblcP#1s}9Vz6$@#2rS`XKjHC5G$DNq&hDZNvnccBYJ_LXXJr z(?;Y0{v1|h8T~d%w5*tb&6Sbu`MDCUA?CHzT9W>6sOJlxP=5z*h&-qC|L$gR=R;Mb zq`ue7#=@Uw2Vj3@JPPT?NJ!M%KSsrg6p$;($LT*VDuUsfF}QT^e+s9a_J1nEOipTx z26GliBZujbcHAyS!@QBi4B&i3(SOQ13lks(PcqP9XUY9hG|Z2GhbtSjfZ8|-XLUJ| zb|r#Kwjd2^$Ggzf9fSe)PxUrp@avmjBPk$m*(eJ#-aT?$8(_Vjw*i{MM!V=M>;G-k z;oY@5EO9wT{CCl`GVKJCH10lIvvNGWlyr}i-<~T(L$T)*lP$d7e6$8+KjKz1|GJZA zb4CXM-So003YgY=A(Ce~t-5|P#+DUP(*{ATP-u!Jt%Z0^tGMT*; zZktb+d-o9IYV3E?cp5FFA@4hK`t19Hp$}B@$13@#xoUmC({3kXmE2UO~u0vq~8%ep-VrAGG+oK~~jmyOs^|afuv+`)T^l z9nv@%5|$)7_(YLx*?CP|tx``+LDJIx5N4`x%A<<4KAhx!r^je7ynifumV2J;cE_6LkTdX>d;(p; zOYLECAj4IoKL9leIzr2<;D+#*6l5jO6Z~#6;ANo3^wIDOQVb+X6 zLjI9AZ-3_Bd)S?NmGl9jS?=W5$gz0#0ZzI5{ykgmtB$aEs{(rfo0_MZF}GJ71@WEb zp~|{g3~Pxkq5w)ALZ<&;uEw$XOw)cBdv^+8nlt5^9=jxC9;kLnF|Jg@tY^V7MOwah zn|^?tB-27UY>_PSpriH=D_JKmN^~55ETHCT3OH6`hOK-qY1t9?gqe?|19N44cGESs zyJdIdHmie43S)sL+4JVh9m|c8DxG|I1=>%;z^g%PEdZ%+wI{xg0g9}W>a>`pOo6!~d6%Pa zVnJPU(X9Lv_bb2t*eob8p87!Nx0}9#PyJLeh@@#I6^K6NBh%@2+9mpn^Zl_zlkGd= zk0AwpK`}B$eN(sRjk3LV=QeKFovkhbt-zNW)m{qOxpaPe@k?BxG9(ii$s-DrW#Ur!ZNmJJQST*-SmBo1UyLa+NT|=hZjUn~xB-IYDO+2;}3~_Pr zl(y^>$20Bm@^p*BAT7FMBLg*ulsF;s@;W{LMLu0NXyne3a5d>g8T1jRe)JN;{-9@80)lqXQD6PlpR-m=D|UkU?`?ee1T$d;P-{QVlZvbtCZl z`?%p*z>LWFNZL_64o(P}pd4G>DF_jtt5{@3*sG$ZLr37c{dxxJnK=oAECWp=T+IQD zpE5SBLCbkk*S<-;0~W|QUpK%FDbXott}|zUYS<_!rd`nw{{aX+7L|v=(lc`Y!}v7! zmlzlij&7$oNK=#l7^C&{ho{{~x5`$V`{_=)deF$n*RsWJl4er`8des*`bROVC3!DZ zPo8AMts(VdGu5`)84?25GO+m~++GBXny2=OL3VsOZ!uu({Fagyb#Y%GKf59Y3+H5F z{dHu}sdR1XYl;~-nnM8p)^q5$hka`ctkKZh;C#|~gf?GDelFrcFT>=wCnz5fCVGQ* zRASX>yYQ=>eKzW60Z@s_qvC0Dzddr+0pzt*7P4OZDONO?@9Cnad#O@E#@pW9V4Go8 zHpK>p84zgHgB-H>Lf^~e8Db3c+O^SE;2AB8)xQc3tmkKV*q@Q&EXhjYxn3QaJ-WZ7 z!s55LxVbC74Er~u1xNOP0s^?k+1NPY@N|G_jyqZ@4!6P zlb-I!yOm$dd*Z*O;Lt|I|L4--DaN1<%PelgMw2iCsyY4Fwi>mqr!wqK+C4EvkH8Rh z?fsQr#ulLYEr>itALbj2in3brvUNoG0NoE6cq~<>WTZ@A@}6l=%N5e{9$Vt_Sy@{@ zm6s4!k>`jMV|?0*oI|Ml78DrpDs@F(0yx5Wkn>xw#^bg$XXh_KzjOKPEgdlmp?gy$ z#Y@`c)F;`>oAt`CXM6?d>&F6MZ>w^SOIF*)vji4UxF-+2sESu*&{QIF(mBqfszHPk$0+_xJ`3jZIR)6yQ)CP$Vp5L z-!2Oi&_jDq`P5F>2YJQ_jMEya6R|Fj$goA1uO`iVjj|pm?aWmvHJ=(Zg#hz%5J4P~ z$?!!^4vX)2$8&`;>db8tH$l#iA2)d%SBP!WwVL)NQ7T_dnRGGH5ppiqGXcTDX9w}jF|M; zB5r3G!vo5_5rarUvY(j;EfDL78;E0dxSCCh1tmgx`#&!TM1iY2ti!ClmWaGW-_gfow<-w7dfjavk%=jLy5)Mf z%_qLy+RkddYaol@>4J{?>kH*kae>K_(Uz;7XaKPyGtt=i!ek&}b8zzbCGY4&?a{;? zhp%Vvm#>Qqwk-(F0DPK9zR0t{mJ~jRIP-=W07E9&v;p9sjoJd(m#6*Yd$u%WwutQi{25ZAbBo z^ZC61Ms?luT3V7TP?v~m!B-NdBHmOYw2#~40aq<%;x zL!sI~6OZfyW+tBRulAdQ8fZXk#({mw;NEJL%OF0w;lnKAD`?S3x_6w>+N`sK6V_Q478Y!pzt)~jj$W`X+>9b|x(Hi)18+EWs=%%yi z>=zH+!;diuO+G96#vogyjE%~#1dg^J6Xckh=vsQt=Wd$^$|itQ9`MD>9DjyNHFf06 zOS#O_+VFKDWaZ(4^+fVd6&|Y3sXEzYK!(|R`02=JA-%C6+Prx zIp(cR-@!RO*C1f);j+p+P`)0=%)#2UnU+pT+`L7x#3r|{02Jzd5^Am%eo*po(_OMC ze29DWAbnB<@)cgvxChLm3yvb^0F=2WwOWy`&H246C$*T6@*6WuL_Y)4cQ{5ysalz> zZbhm9j9t>x(OWqXX#d5#m6LAQHj7fQ69b&7^aKeq zRX|Y@R8T1i3sjU2sR5)#KtUP?q{9K}E>RKbjv=L6S_Xztx+P^u=|*CR0fw`OxS!{o z-|xIT&;QMNUfrM1$o-w~-fOS5*SglVuBr1gc9%>3)c)eo|2HJfIHCBd{g%S7^v*dh zDSv}*ReqWUu8f;D{%l9-z|gZR)ef-G-ib3K+yPt7T2Au@TTjspt?g-o9PYJm9aoWM z(-Xcd;pfqCFxsMJ*4Z2=WF9el9vw-;U%Ch8+dG%}I72_7;e*^!4L3|*{Lx4|1liU{ ztGTY*6|$T#*8F!Eguq3d@Q0@duJX;(&}`o6mbSFC4EYDVI;!S$nLA;ub9ZCqSr1hY z=hI=`2-u{)fum5~s1xFuvCjVT{gtqlK*8X*8gWU7FD*t&eUJ_lxrw!6SA=?hLSL`V zIIhMlSH784#|r)B@^H(zC^v9s``u46*NRrg1h*xYte4ocCOrtRXh_JPQElVGZJ}Rf zl3f_Y+nufg?u@-AC(nXa9#^yXKk);-<3UaHB7tpC`+mP(Km5!i<(vtN)qwr|4Q`V@ zhH_*@KtTFUidAS+Z;fTXsJM`qUP8iSa?O_yilz7k-D_po9a}ym8hB1F>doA@?|%_l9gx3<*Jl#(C&bl5hu? zRJAqL?956cjq$v`{H7kv>bpQdEe`Uh#Z?EVQnO0#AAvKkyvbt{SyzMGM>D%f5l1(GIY+(2GNc0*|y0yn!HD*Fj2~H%Q=|1oOv^s z3IjOj?h3|VKk8F^M+d7nIF#*oZH1}Ks$w>Vw+17rA!W6HlKQ$!Q)qpu@6m#g6AHUb z$K6j5mvCLZI1XB=U-iPl@D5FT`uTiYhN*Ytk@3krC ztGywMs_#yFzUo%1+L?e7U+Lk00A|b!q=)4M(Y>$tE@&0$&tkVKQudMq;erg+%-I97 z8pUtv-QjiSqd}v}qi!es9{LbriLLT-OLH9GAYf&6Nj+4jWHnR})~Dwa`K6?jp_n-3 zK$A*y49xJ^jm4Z<@^KSf`o7I>wc|idpVyP*3K``~S{;C=XdTEVB$7BSTOy?|{X&E{ z)cZVEecIUYcktpyi2B$Y(^jOFu1O?fOWVEZ)fcE`xJuif?R%7*J0|nJ1#o|2YjV@a zUmGqsU9214+TdZhR(C^1b3V_!v3h5n!?A5Ma^b2hwOQ zusni6-+S=#Wyg(~@Q0x@w@ALrhjhO9l-dcAxr!`;ay*z%6iRbyjO8)*s4_%#ixg$# zYgfyWM2+vm+qg&d>};Z`Dd-m2%(w}NG5sBviT4MVWMkHPTd|qbW)c`)Yoi6@FQR~i zYCfy-vw`b?C>9@cC1UZ*h)!!C|1xBMPVm>#`53K7?>t$W^+RDyKo28Vt5Fcb#zW?U z?vsFrC+<8cYr)lSi*pF*d3E~(VW{znog^At_T>Auo^?is^NJk&YoiwM~Z z7^K@#;Ra(>Nbbdo(e?~4Hiv_pP z1Ju)!8?RI07R)PUjW)ME*Svikj!``N_rem;6rx-Su*>OjPBF{bU3?3FLmk-z48IJx zJ#NpNLf^M__?EiNxI5Nf)}+Lctq_MIUcz9$ddfL(V`0o6+DkTuiBwPn=rz<8F%H%d zSCz_ytK*72blr{j;9a%I@$1Xycq{(=i3GrtE!pzVU);S|{JrVsbSL=Rr2_Qq$I?#D zF6}*gPUrMUaP*Sp1vC+FlLAnTmm9iyW(}nJU_2Is_vqYq;)x$u^ljQZ!MbBgmbwoBZR}Gcsnx^ArV> z7jw?d1yjkg+#THrrbDXPS}($O+h1_6Jj*nDoyrxV;?5TdP84SFN65W%dV=mru1+Xr zZHqj9Goe(7rzRmR)I^t*7$Kh<$eL>Hj_w<1DU!b;=n}{NPHU@TwYor_0G(%oU2l~$ z8O#bEvyU28-Wqo}NycyBDl(ZTtWb{NfxH`YEO1W8*fIH=GL^4$T)A7wP@Sn8Gwo`I zL%Jvv>Lgww=$Lz zeDU%2aTT05U58?}9eRpDHt6PZ6nJ5-ORzM&m|X<73~Yuft6|NA!JTH*Ve$qj$|(W= zIl=bRz$&i(q28XkU!YoG<>N=me{$Nn&ynVS9#ln%Y!6rsnTm&4c}^tBZwn_$)BJj^1=CE?p4lVC!azP(M&~;St$^ zFCkFI4-}?!P5O)do7df5cLJSx9kE}&^4%cH z=U%m&X?U*J15xwi+tP&CdQ&12c-vqX<^wMHl?_)O!RKOE0R|s+({oLo2HAAo+}OPE z`~hZ>#qQN(%1ONg8u>sw*}etOoGo?7eE-BrB%OO*ak5a~D`&;Aux9I}?}jfdx-ve| z1eQPwX!D2Mx`?zezag$-Z|1sApryE6o-0l~w-)>`=$M^avm~BnrCf!cVr`9~Wl*`9 z#(&}H3vix&$vDh-cD~s8Rd9{NjW@^LK6-#6<06zah)^ry6@fxnVT@p3E6Wd8r#4Bt zY74I>Ux#Y=T>Au}x#W`YF_t(;@xju4m%k@T&$~?)#II?_} z7kh6sO3kOX^-RaUZpFp9tUoM`THtDhPhH#dDomwe<+&rK*HU_H7yz{>Wd7 zgzYt4`d!9@C1&Z%_I-)0=W!6_S%Pj+901mrJtSq$6OEMTV=HUEJoe+fcy=w`E(B*Y58KYj0uFVysN>=OS z%yz=ED|jMI4>*%PmR8ul8~PZGxuD3mZLVRJQqGNN(7cQO^ru6y)0@~`4(bZ?*9rq! z64vp!hj)@Cd9GctGrf;Vp=~E{1H52ZVPBTgCgM$Ou6d}T?Gjf{vQK0;-q+=#ivyjh zLlqZw=$1#-s({u>+>_}R9W+f^sX)J7=1{{gKJK@gKeuG&-DWC_>*QztHo)^+#&63nzAc|mo9JHk z5oWF2(puo{tAz=fQ5RYEpxn3}U4Kl@y&`KM68HX-e z0kaAkE_1N?PVhcAhC7?7 zdLO}LJjs^Ki1Jf0XE-P?ibS=gIVEMQ`8tU#W=BKPW?w3VRPHupZ#^^2p;b>Dcr$q( zvFNgjmtcs>82g6k2$3L$%d}J3E`J}GPo~UDlf9__j3^SOO-KE@;Mt7|e|xrX=5e1C z^3C=5^MRyA;a2FCWsne?w_Ug15WCGUty7u)gC7Uv3O~p7t9u@?LQ*l`<7lBTOCk%T z$}IWs4+D@08<1{WA(4@&G`eeJPLd>U`|0D(0)S}e#+n}JIu^)_@Jj3rEZV_L2 z=h&d{g4l{#d~Xi5Dbp&}DItDo&GOB0@SfcY>W#%2n}LDC;ukca=WK8aSyUcJ5fKnY zIw2fwhTa2PvLy@CUU&^9IfF$JzdiJkXBAi$97RpZ%PP^8_Of^0ZO9&yEtB=OJvXOr zLyk|OqaBCDH)v%V9P+gpORVmqXUms4+VjR{eH!;ChSM*ayeE?IBC&hQ*b*hj(w zq!~#x&|x*i-qx>VqRdc(M{8vnX=Pb{4es|YP%3~ zHYppkC4F3KQCs3p%|kZ)FncHYd)j3yWQQ2D6v&UT4&*?@<$)`63|E{Nmj?_$=MLd~ z$Y&;@K1g#iKksojDA?89JQl9_Q9n6bnI-gBC!x8jK#aq~C#)z4JPcFLd9g0znkOXr z3dBVPb7f9%(Q(S~`@PJ8L+v7nda^B^}u{;f_}ZI)=|9Lkal>^cT6_Oi-v*I#hip zU%|oDj0slH3&Ggm^%U~Cuar1o5r~KBs=!9N37)82CV1jJw4kNuvna$x9G2u3!KgKJ z9_z4Z8Z?k5JBA|SfcD?9h=#mJK#>hc<4C^HQN7od)0mZm-=-0^2e0~WbCRWZ^pFq} zXAIO5`3a%)whC8=ek{p^fT@_F;lXP4G&wkhQcxxNWxEu=DZN0rujgW$xQNBvz&2(1 ze8a@3{+Cs#N3^rc(NNScmUsIPVRLy2-F%}i+Z^Mz<{!^DRQTUvd3sP!p4M7E8*gMW z+*QN7>|nK;ve_a(k2K2nUQQ++X(V#2U0>n;v$f1@biyW4!J)V6fPm_+he{$dkE+%w zZNHw_`jr^G<=9&B+kx*QYU=j+{^L-w-d(<1j#RHgUOUnMjWUj-gSiIGI4B}?Qul1V zrD&Lo3?GL9a@0>PP!^L{PJ_Ahh;8ofxB0&0c?F>a4bG1$kEtR-Cl~w2q3Rj5Wci}Q zG69c>M}^DIpUc6}YX7pw-Stdbl5_{d-$Xd5ZQL6ujZpBvl`!-O%&VL{_DA2Z5$0hB z(5HA9PAB6YKLoM%>OmqL$ycA@`+kLSnDWN>%aAdLF=_nTl)kaaQQ*ASWp6QNKJ+A} ziZkEOn5EFkaJp5AopKRht^Sig#e@7R1d6Gp$}DJ5b))VDO(i26t$%na*KDVjxkci) zkJ03=-9R*Q2sRNm&_0<t8=g&TbS-K&S4+3Uf2g^m8%nUm$-x*D-OFLQ zSNx--Xn4$s1;Cr~4$~={D_UmRN6;BYq#_0$+cv}u^xnElWX8|`01mnx@N=kan^p$? zyiB^&O(b8gAXX?pcZGp`qeXgNP=W4^=`3XPqQWR-GgCgs(|s=>A=M!JlbxM}s^i|t z#H42!0efNRlb9L@*GEMRhUibKlc|--S1+=={|kHb;2^-a z^}dk#$em>rkF|=tZ&zXqn`h>!3}og*4h zF23iQp`2kSMSl3!O5Hm9OUJ{eEjN$6Dhx^=q<7#aR6H5ajsB>tLT^!HwU!X7MIMXi zXkZV{U6E#oVeUn9*ry3;(8>HRbkyLTjkP{-9Q|xq&luVb6xo_~;QcEKQ zQboMlxrRI;yuu0eiC%C=`qbrs{-2BAa6@d=FPK!Glq7tN?jip@QZb8ijde&ge@F+9 z6mbXQg$iIUVrc=B@&VMys>|o5z4LyaFw5crH>$1}TwZkdXZ!9B<7T!c_xR%@mZ^9i z$OPU`<$e!_L|Gvy%&qn{1S~X+dEI2>j_r4cWm7E{1?RWvj_UFoH_^+)o=$p+B)ExF z92LZF*mVuB3q1565dY*x^c)%(KXShV%Rlj`^hM%1A=Nx4JFk&Cq(d0a1=k`b%DzDE zsInA0dY$h-JlGy9?OggmCfD@ZbFE-&Hnkse^fuhdc|33DD9SfOS_L6{c?b4~`x|A* zj7*NonjO`>?#gF+$R<>x8(F4Sw&#ez@JA73`FN_D@3Iw(+2SvCFvR4&r8TuhYJc7} zp!hEkn=F%$dVS^pmi}^?q13aKGbwF;1C6#WhJC)hrF}ar5W1q1O;8%)^KY|ll(0)c z*{8Tltt7&JG&&O&`qBl0Zy$QCzP-6b-n$NpY^cUagT(y8ze~Wd?juNd}PBkJJ$wwf05vzXZvBx$gryX*z0V>XK8Tb_q`c694Z0H5wcbO z;i7V2V70t?i|ods+XReYGn~B>&exKevD@52_+f**aT*l zUVOSjZ8V#!S}xY$NAso;NnYkvN8~qP(CE|_33V#RjTFvAwW1_9O`@r*F$xI5?pKnE zx4`-6I>WyGU1;yIh*Q+%`O&UYSxIaBX9oQ95Qzp2+LaOhs&+V!w0FRhpSh%*of<+B zox&+3)WjbK-Wob3$$CweSj&4iBJ6`%s`lFgDiRvkSo|VVTW_LvgQ1$fa$2Y+z?h;g zMhrdJf+&hGv3efS3q1f?=Pnyi@?89Lt9|wRPkNJ)JfB_(Dd1TVm^!D*U_R^N1Br!q zlstxonmX(bSWrnTq(lEIh7Pk2Tjs~Z%c8b^d(|$pl_`FyaZvF;;pP4~>cV$XrQD>* zzevkm`R$u=kYx$g^fR7FX_k&+pF5kN_KS5NwXgDN zziNjPhRUX(85@|krIuG!4m?ZcB04K+nc=HjX5u{wk?MPGM7J&hfxZ5R3e?I)pSN5m zC@T<#>vTYhcTO)KeM?Ji)0C3)U5eHWP+|BmX6IYTB3tHzzLQU#OI zrmXv$l%YW9_{P`4I@`Y}d~^5%P5uX|V@4Kx#`z`X1I;&CqAlDG8-F`Fu$aHkoV-Ti zZUf6w;{Nu8Gc$r#QGJuB!3kRl5iDC?+D+S}%?xBMvCbcOg2oukjhB>-nAOaWcvGnp zrX6#4D@q zU`9LD<9=!&MH)dyN`^}2>z2Q7nB0&Eu88 z&Qs{PvAWknRSU2TSILv?%LlT1URH5NcX26t+L7FN>=0ibWRT4J$B6D?E|b~I;`5TugReyCSaKKr zZb6{TRXYkaXfUWhyh!lNR|5bY1!^=lRfcLY?2e^}8Tvc`pVm{Z&^XNh(2nJO_tO1W z1u;?tj0V2gGS#XV;AZF!QqXY$QmO56Hogavw}gIYh8s($(F{u@j&fxNl2{QE^L57U z-BYXBx+^p{%nz}B+m}}G7EHZYOMBaCO;zmy(77u`t)^8&o9^uOb|YaT-`+Qq+;Uud zwTDIajN`Aao8UK&-L|@$ZuOaA`16;q4s=pq#nzMs`t@v)`6ITreItEnO$)7RtpT*y zt6p2=ss@2=JrUPMZ^Fng9+bjCGV~vI5ezRLguLkNO>tu9I!Kt^eUYAxKrE;@LQKcp z($$gBj1@fPlHdyjt{ewUyOZ_RO6jIT3%-P2gsWE%C8h|;#(WQnSM!i($AMy1cy`{d z49pdM&i~hFM~>+LC;B#9Rm7*G;Rnxqcxk?MnzDX82lfJUw*F&ONlAn{X$NdH`P!_K zhdo{R%#qI83ORqPa-w$Tw1xY;Y_l|%<85nKnIq_CI)bkV?I*NpRsR&%9ZB!kGfHqa zg|WjCGxCwpYlk7Jd<<;RhkuVT+!}3q_?7vEt$RwkmGRfz_CTw6ZE`FN_SL=mtkvGw z;qn3PwQyE!ql}x1MudYyD(R@wlEmS?e#P)`p%zM`z9rW92lATfjN6=H-@-EzIy%K% zB;@gW@@4v}Oaxf^FgZze4xwDhqhn0@*did;n1cEfi-8wSW!w$)Kg?rnVC;{YZ|HuF z@8g|AT+fDv%w5?fwQQ@GaSI8R&zzV1YK?)D%JWYWyt266RA-ui^lr5qg3-DEtobcT zE5*s;wDfx_e5{1{-SGs4a135Km37admXbDPf0m}KFF;Dr{dMQ-;x)@C7ixDI6!78z;m?)bFpf%O8Vn3K#V`X3}QM`&|1YeqVToR_XL@xu^ZS zBv`slLQ7@adUfQAGqn-gZ$WW!L!KKXDG{;#h`?qJt6ysj-i zf3QXd2wlHbYdEoAwU>JX*0Lt;v5RgGALR^|XdPhxNFD+y=Sp>&FPE5yt$Kte8=Z73 ztFkeH;TzKp@I0T@I<4g+944#zc}nzbiqZ!-I}q`G58_ua=W zkZ5kJpSJFiGwtxk!?TLg6S?XoUl?7o69X`k`||EpFRQj_39_az)mqj?8FkL+@`)B1 zBnzDjvpcpYyl^1?=frG-fKC2(dqKL`!A}Q_$!ybY@3+6>6x|bTe%h8fUhEZ}7|77+ zNs*>=&JTq$RV)=&WQc1{Br`U^1Av&aj((@$mJbhljgf46>#(AE?@n#jPQaRvN@ec> zp`fMqfzch2=k5;C4}^9H(p%Xi>F5q^1GYcD-*9tw+E?{IY-WUhsra(}#NWFSLeEmY z{>()M8|v;Fb4xG_%jKl8(0bW^f3ehIV!tWayG6|?ezrt|OkYv@?}FY!Qr;TTZ=i)Y zxTo-=VGNT^ouU96cJ$ufMqr=dlL@(KuT&p0t9>{2Tq>W1Sk2c*Hrj{3RBpsBeA~%` znkHpJ>qDj7+zAt~8I!LNPRxYkPD9U}Ycs#fS+?Oh_bwd@D)V%2rfiIT2Z*MY-k z07sDuDIYp61skQf?vbofpopQ5nHTQYcE$*DKq37@b4d&h*FuS~<4Cqn{(LC8%ScI? z1GUm-;J`0E8K>?LS`J^87n4H%ATavRrm81!g;7`(8usc%F1$%asfxfdk~*}l)(lE4 zzV(t#%IcYXL1!`LMaJ2WK4H$GMY(P5GP1j32G$6pnrEgGP)s#_F&j{pi!#V?spSiB z2eJ%xeg<8;XjOmL9cj=Xx31LQ9<@)eTwfsSPP3)vJ=`anJfAh5Tc#b@NHiI~bLE&f z8gb5wYbGsT8?v{VJ!)(g9~`$m)H4BxWzHs*jjrcAybt)=#K|_wX_&ggg{hL9?0nHz zfVq>ZPMtyA*1y>TU=vnRiNHzl|;&oer_{hf+QEy>s`S;1e{MI32V{6CIkdXG>=Zt&#QYmUKBknVU zjJ{6sIS>)`dW#zO6EH!P9?r%iSc@>%b!nd=v>xSDMT36CwOghrhKeuu6>! z+c9=ZLfo6;aRYSgnKU<6%FX zSYI!>Vs+PO*&L5+8hyAJNs-@>M7!Td@J1}Je9J0&bFgtuqW6v%a>j9TZ*jSC%APz{ ztD?2?(JO5M3U>Vrs7dl9;)=ekz-v*rv(!p%hpvIWsO+&m5+fg-K#Z1auWBF9y*{sT z;sk#!I$D19csIkJ@T;2x&n>>E%}CwPx(~f5tLvgg<95^y$zak!NVWCVIdCzDx;J`3 z2X=C=Q9bwTOc7$3L=(--xsxprLplXgL;ARL`1_a@OnDxndH4UmvT87yZ3r~O7Olkq zxfJD=e7UE($tsKtb|GEN>h5#Muoh?hV0E%k3TQgzCdK+GPnjq0~& zKr?efMlV&iGnhh5N75kF{f-MfV22OPsbqRW>wD1)*5PHM2eYFj=w{N%zO*Y>h7VQu zm%nhSyxrHnF~qmHq&Pt>nGy0Lhvb3a18k}vF(AI0E4j-*DvfdI`Sr=^*72y6jwF4g zp859ZL6<})8MV5RK@3Iy<_lfrtSXeTX>l3%OX)FNAq+=;nv`FszU&Pf%`O{k*!&-F zn)^nv@h-V}qfu@}WfvtTCduz^YQX2EVLqkY`+oI2Y4ABDjD77Q>X9YgaiFCiXAEkO zg2(G<@vuSBYyBE4LAiwHWUF~t7rBEyvly0e>9(}nj3p9^og`$u=B7pcdI9{!Nm=NQ z`8DKhyt@MJEyWGHuZSCfcp+JiwczT!0~CO7lWRK2ResT4dZ{F(WM0UO)|i~zFpQxX z>r&T1yP_AAtQO`OKcnWj*8Z1!JFduQ&AoDth23TBV8T^!^X~x~(iobp;aqX19lq@q z>$+lL+PY6VsdT@jq9;JqE^p+TNSxxK6cU5-9p#zpGfUuc;T|b=-G^`DV=Rz=QtVS` zTLce(^sH&Mqw&yLh&Ov-Z#S>X3;u<(r+&2?NE35KKl!%Nx#zX@_qmv>vw62sLjMXP zld&Otu~#T4IRFzL=}|4Y&&3U-rf-zs@9wPzZbQ|yHgbO;Ky2U(Wn&U5@!d6q+bTu& zyD8^sK0MqGgij?0GWN&I*o^}R%j2CIzdN;5n>df6|3&j8n|( z*DkjkX{~iDv6<;K+Iv?Ep!l)Jt>PNoeakJYyHCZtFy}R~F-;!!^2Q4+%@r#xJjT&&2L737R#-%d}bic)*U5z zsW^qI)}nO}*M%9Z3NK%E2G`#61jvEuK&@$6w}yFz9P?KH22l6=vU+|Cq~ht?4Q$7E z&xPJ=I9KH>i?9Js#6{hIk&E9p+uC%XdHaw@WCc+%LmaRGk|DcF#0o;gUX7wyb}nTrY+}4WxO}5|v4u*wEi>ZZmE3>)yVbEO`Bz zIgE~sW@Go_BTZfVU+Pva_%z29WMo7RyG#mkJHb=by#)ofu&BBWoS|vc3W^owAS1rg zv?|ou&VTBS^f&SCEy5_Y2Xc~BqrTL3z=c3ZWf_CwAqpyok(iedu z=Az0CYsETQBHNivTXtEFmug2~#XFORg;g&1^K(1xm120-QzH{1yRY4yEQrzfS@Uu4 zQ9Rg-HE(;Alsnqwx+Wg!$lqwcA=j645b;>;ahcQN`$c4#VRmP7pnou9yvt*^*cEr$ zTL*N5E*p8HaBJ>5auWdV+zmgG&7hpu%)nb;CB`w1bhQq-K`{xL&KnAG!a!|>}- z)d`p*NX_gQ`$wGN9e1Jfch`vT0T?W3CvPolF?veIfV9Zu^Ze>4LmK}1%U{T49qeQG z54NbOvy@k3tNtuWGX{RC&dh+Oq8a1e)~9#)Ugyg6HBg#b=e{oZ`Fc8Q-ew@x&Fq#S zKosuE@Sy0Ist+9@w9dW~?qNNh0uLcnf&mVbvMP-{7B6`}aV12Rm9Gkv&(E0gd~wfZ z3DY-YRg%kLN~*)vCpuo>b~gAKK3ZK`e?QffU-BVSepjakKl9^9Uxb3b^(TJ()Vne= z1S@mh@2e)dDdwv2`s`NdrqE)XeNHbN)&C8XN z+M?J-IIbdpluPR3P=5?XV);YRK8YGvPR$CU02e3@>Y-s2tP zQtgmFF6*vJF$cAK+cY}|Lw!)!TqIYC-4=YUF?o{BdCF|LK#RPQ=Z5Nc<64Qn*Gnct zFhZH^h+J{lg+$V2>jOQ~&Q_bp3}zBH$c9iF-UoAnHZt}jX&3`48AWNm0$aAp5@^Dx zYpT*>KYRP0^Tml|?%yxruQH|(U{I1BtHbqquZ!W>Sr8sXY&<1~LvDYAcC5wG(?35= z2w|ewdAa@7G97yChL(Zsu4*Wzj6V_*otOEdL8q{x)do9ZX3LVsqle)!f#roi0?DOI zi%jlX_0B|5ar`8M8RMWMS++jx9qn=Ae}7z|?FpM++O9%$!&}nriscCR<{D2f;E62H z8CnY0dRXi-RuZx_``EN1v&`5J=D@$mS}3_>eft*avI6@ImkWT8QTeH=q;dST1?5*a8Vs7@eYK1oQoooEr zDFY#9?cF5UyNcV)sprzvO9+P0H583?A_XJt*Z93lP81h zfb?>MekLOCI{M^)I^&rJKIs2ejoMLi?VTZ4(N-!FtwgDA6?4cL@8EUxU*ZH&Orz;& zK-fd+Y|oCbn>7oY+lMmn%bSj$?feGLYCf&1b@m_Eu}~pIzsje{%h`XWUTn+^hxM0~ z?rk7du4P-Sto`%-qyXs=Bb_9(+W$v%rjV&d6tU!F&2zevL^iy-VXEZDlH>FYa2&P> zK9_U7bwy$OWzc|Xy9AZ*UQ(lgzB}uf9sr+J5hBIesJyuxzD`@@Ab|P zBBx{bBp3b$F%vR)G9Oa~0O(?J=_=5__=1~DJOaFh$%w96n>paEAot!3>iWhn66vis zCF@N!r(=8G8FZN66V~0AxOEg62_ETCx&PuP8v|+g1HDFvk^78+L}iacwCqPjF=%Rf zB_$Dn%9ylNWA zQu7{VFVoJD#IweAW=>8~-s7EzT{Z z{W_DIjUJr7P#AdQ*cfKMQ%!{1--S>l z#50xr^WhL!M$wb!&L_|I5PkT{em z0wIK#L&@eup2G$AKgY#mTz7KizSEmqmpl*e%Be#&EqdipMC!1 z^uwA`aDLwUcGI31mHuk%8MdVIHz;qwT0iwX_aWjT#8AOG348pmUEbjb6bO1_+xJm(-aJ6HD$cCxYysbC#1<@mi$ z^jXdl0*~W;YGQM|Fi$k@`83Q5nhI2*s(!#I|9U+Al@p`Xrf{69^1M^P8AXmd?&NWv zK_Ge(&DU|fJ++#gM-<#zQNsK9IAq)a-p~u3qxnzz1MM|M#+DYQ&5iy5BHu6fsB(+@ zX^D^RgbV2@jK^LrsABrhhoxYfj5NZwPW{7k9Gsd z?afg~pS4t|uFHRLpT7O4VyLjb!`f%nd!)gIg;ocZC6hnC-tm!O$}N!;Jzeq{y!qn; zE8iKee)^#Y>@02f;LX&3L=@?tf{m#i?Ecr02A2|CRu*-lWm0i;VC|RwQ}OR*kzS<( ze?uhr56RDeE}L=0QGj(AFp9;VgmaHbFNjo{v0RuF z5hs6KV6xXppkM(e{yz~sq%7AyM~PL^OtykB>;31SF#Arc#wJ)ww{np~?(ih4)+hlp zl=E3{QlA8l9}k4vp7<4YM4kSDI4}!y2bJrwVM^Hg%F<{{W1UM?-0Q#q$+%;3RO z?q@F3o$OfbHTs#Ew%5O29f!e%i@0!cUdnHP>*?v2FOlU~{G$c1E_GxD7g|}!sUp?c zbOI2%du@K5s$E?u2cbLY@GPs#X>64Mp2ndlE_wW}8H~83j+*{Ibw$%3UkWe&4~16# z24|nOYbW7Jk90EkRrf!tr8>un+34rU6A>|b4-YP26Sd@&+b5eNO#coXQcp7Lf3){% zzJQE|$&d2{!w9{Avx%IO&)VI}g9-7LK{lxUzqtkU*ex~`JIn^*^l3n1RU*CRoEa;5`?l-fVWCm6<6=^+%W*jNs{Q*$bU zyVSgc8hLU;Jm2B&(%DK5)6+kY2ljThmIyqyw-jyxqyw1=Yc!L6B9#0MOy?n|VOn_f z!f{Rv&orVx`EWc6m&LcBX zI4OnyTO|CqNC2SIe~W~pYyQ7Q0TY4IvYfsS_>*!80qc-WwOWk&Vau0M{vsGRRA4WXeSv)C%bAvj8TX z9uDieO^`z*cIQrOcs&v)S*Q?qmH#d%4j z0QKs+4j?eIOSUUMXYfRD#ic@I={4cROx$|Vm*2S)T>TfSEu06MO|2!ANkSKXUwiQ2 zv((r~X-MrJI@sKm&lh8*oh9Fw^=Q}ad1_cSo9K)yl2ysJ1-uR6Q6D8z`WWf(?7FZ? zixT~mP#9Xg{F`r*O+`him9bM6$g>|v^yl6cf#^vwi=FYTIYyQVxo>a+@Xi0~Oa32M z_)Q2YzZt!`t9~td)wSGmA3vd4-a3*yi-s9jAHTmkep~BmRxwzK>}}z;+=sp8jNq46 zG=RZM>&xlzE)alEpIlHvfv0Pdfrh9V$20e<#<3pHJ;vt8&la}+@k&W_6lw5(qFt%L zAn=-D%#Dp@lKcBr@Mv;e$42nZ368Q3O_cSHV4oP)GRTG@)TkE=#hIo`_#q*4OZEMz zBfO#J(=pmey-*B>WVp|rUsk;L)4fPB`Mrkip5hjUhkdO&bXQkTivqOYJQ;3)R}Wpc zwg%fx55RrAJF?)={#&C<8XT(oo`1kt+f*EYd@A7z=dbH}f_-mpBH9YFi$kd@?zpW7 zuJ|6k{AVPz)b+;j0 z>?|?pT>;1C1Ujcf#i%l@eqK}f1$rGEXmN~-7d`BS zGxMVzxZ~krh67mPY#3kRpLipC_QA=_Ln?r$T`FX{4K!G7V{_CctSVs|DAB|M9BO>v zc8VH1G9^L@9frrA>-R9j@TrrZ2e@NUYfHE2qF9OVdU#|WZ@=Q3g}WZYFY&y#SDE(W zEA?KsHx9+E&0kq>h@;C~_1VC6sADw2`^rMj+2FPVbTK1t--^8L;n4%$EH)m(7bu*8 z>p(hTCvL;B-(wUvlX|bmtMP=W#Fq^7dokuuNwNa9xzLq&K`As3`mbbLwc5T z>ZP#Tjan56#bW~BQ+5bq!ON~^Sr-1?OCU?%0q6AsXY_IRj{*M3931zU8t%QVh>#v{ zQ=|9%`-T(SqfDeiw|OB?bpcT6$q70Wkqp=SfY?GhF4*+ibWwtD{R$4oV_j<|aE2Jb z8KSxm8XO*?U=_Dw9wWX8)*Y)6OVkN9T+gQ7ij*4}f@}6?Ao`U0))QJsLlT%Gc}^UW3P<9+ z$5vG^p$S^6{x{~T_mr(_94oj=DU%n|{SqFMHMqR7Ar8l3R{v@`UM&XipZL)P;H6e- z;H)rrlDLYV@%#i<$d&Sg7qt1kz#p)-vFo*|cm@b^hfn4YXR!ZN~OIW6s)TXnWC}UF+Zvv|9t|c_ID(tcu^7eJG=h|M~mQE zm+DblAYev<^ESP{)CH&!8Q_k|&AU(y4sjINh>s?vlx9^IZ#nL=&hvgVQ~#;pvvk9v zJ0N}2@4?hzL1HnT`ej+W|~4X(B&DLnn|+|B1yt2j_~7lejxc}rYJ zL|^F`qE=4#hJ$C*cxnYu+^e7|fyEUOAM#;qJM~c&MfJ|RYTccQZ;<*s5Bbx=jfLDX z&m3ng$U;iRar_cs_AB%6Z>E9Y@c7qFXhSmoJtxv2ZALh8(B1Fso#$8xoec>nH?;37h3zx4Ij;AWZI&<+72?#FH>d&F_5 zpv3jZGvDF?|GWu6eR?{At*3_klumQ~553tBCVAJl1QVoeOXLm1`&-5j_FAeKS2N;F zc3AnG3j`edzJDU(#lfZeBFBrr$?#DFL`g5aCn(N9_oM*uRZ_;pQxn zWx8gCzN~P+aeJ`bs09-W-B*A(IhnRN;nD=1qmI4BNzcKv@|rP7op_f6yzVPm^#yW} zH**XGY>&W-Z-*Uppw!MUk7Or(_3y~6%cVx@v2x_fv&zeWkTUWEoKvnXe3;b#54qF70aQ6A+(5ECDRv;T&?<4#a3p14;u})`xMExSNa6ZLcr~S|Cv!3p(IUg}9Vz6`JzhLCnQh{}gN= zIQJ<|K&*7EI$@QyHy~M$#Yt9qzp|waD*Ex!AM6hGHZHXByw-F!Ukj9(E|3|^If%nG z4CvvbJl9JG!12aGwK3E3M@)9iheMqVn!PR%t0K=Cl64$vwpm&c zwlin*#C{TLeM{e@M2JreoOkay;Cs}mbqMt`h{dtJ-h-p96@(}TM4?h(+`*6(8t@ug zV9kphQis+YvV`;f9np;I2ICiC&&zM1Y;ATmp95{P}a+kAw!-grN*N6arrr> z>F?N5Sk6;7=rWc2wn?W!n9J%A>qkJWE5#YxBI=}3N6$h42FV^bYN}1}NAxjePkC-= zla>UTv>Ek=*LNptlaDa~xA^kHX^m-&@B04?>>d1bfJL^ z0+%tlm$)DPxpxh$jFdtS?6w^4@9)e+C@M-AfulkX7IL<>8xVq20nyGdJhTC1W3Rx6 z0dluSNOEhKB`=4J1fcru_lsI*nwiSvH*-U?z9l+(#=4IEdB~&by0BoW!0YS(QqfT% z02~z@!GkXNPyJopU@5VdmKOB?;4x$pbgMmZ-I>zgU7F5U_<#qu&R_tR%4Pn}E1C9&qWFt5Si@>S8tT z{PE#xV9dV4@A&y`wY!jZW?(|xqXfudss+s@5Qv{1UEE~AQ*k{}&&sP$XJSa_`zwl8 zRab4z080EvMU6b|Zq8^vUBa{BV3DyXEg#w8O zJy_$pq~~x@n}N)Vhup{?a`!4Tu;HH=3kh%mG=n&OzbuIW)DoZ;YAV;0*tIj8e}990 zD=>ezU%Kp@=z!h&;t&ZSqtQUAr{0n29NId1YNkFc3v>MT+TNQ~Dq#Jm*_QahC;c%*Iv(+&7%pfA(hJzTr>$ z%7dXB+i2ueIWC{qB7%FUUPCJa2$=n-W0%CyYLLGgQqG&;_(8_cJut*Q=h1 zMZ@6>j$7er|_&CStaZ@O* z=P&@;wL5Y@H4r2`VC_fd*;OvvNs&+_+yeCx6~!2{Up%G^?>H z`6k!SS)5<@1s$Cy>JkB?OiRpw+6r&CftN%5%}=b;2hN4w%M<7Va%Hv#MX5{R@P2445AGXGsd*U~E zpg&%(lsh-zSNJ*alA1Ns0Uekug00RKP{#91qT_VI5I@rUYf{#lWx2X_uL-D*e7%v#FN>Te3!s79wB#$V&*C}Q{_CG{52*!%59FiYK4snB^8sA=t>b- zi|NlbRb!AMi7QYG@0$&t`{Y!X1TOMzhMRWv@negDlF04t0drX4lU0IV zy425Ek=p`L@dI8M*uLEpg>@MqA>B+`QhZ%Yg4kJl9Kd-s1RMYsJ^5!d_%S6(P-K<;&|ckJ%G=;2l6D6^Gvv1QjX zbFL@UpN;bkn`cSL*}DPIW{JMuw9C8 z36mG9a#|jW1@Dr}6B5q^M)*4uCJkH)1IB#X4AgP?L|FK`Knh4G;?Qo*pMXAte)z;P zUVW%`vA$|N_)z6|NC0x&IY1}i+1|yh=qug)4GzNRIQx5uJzvMshr#BvrJQqsy6$4I z_z!S<^0c3VohY-_pgCCf=;h1XpSU%I&(AD>+frafxsN0RJ%88vnE%cvRljAk6-o=v zFRbtqGkua~u;IPmG6BxIc;r`z)T_WH@mPV15mr87cZ#>h8~2|B%9H6~V`5JlBoGRA zazKa>YF}%W{+sMlQ3fHtIn>Z<8|w%}08o;R)j)Y^3`p^FODRiW#`dFo`?NT!tOT&e zXlh=RRedJxpQgB}=t(?IO9UWDNkHLR;lXhO)^IscN@4>$Vt%~-;A0sX4JuyIO93kl zZX*#0L{gz*`@Mh)XOYb*B^9_)S`}a=N&P$jn<)}3jGIAvF>X9F$Y%b;IxGMWS%)*# z=W*fGH=)UNw~XseY%*2Ie|0y1N-bzWyrZ;zQXmbvXbwEjir%;LsUjQ8mu3$)m=z$M zc>oSn&!{5iK>k6awac)W_q*t1`qbObM$=5?MEXNmwdrlSLCkFbgHNCoPR_)r*e=C= zA6AEY#jJ7>L67d9z_gSdU*kqOeIA-NufI4wUnK$oklT#x=>^rzVvg0y^hWPZ=T-N& zw3SJG^T1P8m{`$3$u;}>RFtNjzu+m5Liij4v{hxorA_x#2DZL3g6(i??tNo>1Juv% z0*P(9EDHjv9N6U}Zkus05`-5HNTh-4scfvT4txnHD84g4bI9MGio@R#@pCMPW_E!Z z;5Au6h*Os^ErJ(?E927t7y$_^nH22B?m~^;SHrK9R$~-BIuz(L8!r8La^XiHDjw@M zMz~t!oAYNgoTzZq;RR$AJWI*1e?tPX=Y1U`WDsG8=f1as!nN* zWcx+U&D3Q|z(oXLQXM7QLctBDC1?k;F3HJsjhD?MU&F8$jkE#ZyMf-3?05DiV;`8T z1c>WtX76LBcYTvt<|x0RGb!aBW>pe_L=#3b{hJ}3{+F!Ccw3)lWI4RloQs$E;qdZ& zzq9|;Gj7j`9_;}CUa%tml<8~B3(FGz%$y$LR_VJBe$VOOf=p1~HZtJQb`IcV2A|og zfuv*-6gV|74~;GgM@NF7*$FZqXR5cFDN2FRc&JThrtFHHAY_vqWWlo~JH zq|9?QTa7-7UGbNL3Xv>3WEY02(r=MxQI1%|Y*@Z;-Ck(bg$Lkw-G9EfYioY1cfGO7 zw(2p%Vgy-%Hom+R?hFnS$u@Xz^7if9dlNeyAy%r0CE2eyewUM2&T7?*`*yh2I4^!u z_KTO{n8ltK75nBJs{Hd1oxS1#`6c#>(56s9W0nNvSBJ*pa(IV8Sy|U7V}|0Q)n&!4(07@N8oq0^mKoyN>v24Bnv0Y>QCF9WewRLA~G`w5x3| z*bZ5)hIa*#>bLwz_Sc5OAYh)xw=z=#0SUPu%-=(W#1*)Yy9<|{Z2e`R3hqbx?6J#= zD#Hd%kVEbrdfLC50~KExFn;0WaEt&W+4)HCdpBESD_YqKov>%M(d~%wuX4Jgqrr*T z66xHsF8>0Z& z&|iJ=@WF9q?^mZpZ^bP{3!2qETd)Cm|1Xv)P|IvUf)}8UlmiVlK^|BE#78%j^JzKG z0;ie)InwhHAoCFvfC9&G_Q?!5*F{_1khSD`@5d+coG~fuD^`lVm^8#cW_A9N#wE*v z3r)>yyT3rx$Z>@>4!k-rmx$m$JR@JEZscaVC=cPt^xKO)uRJujs+k&!DXn3R%S$*$}eX(8Bd9xvrNu+$EfpvIbZV#!*(xT>A5 z+w#sYFjooqR(TQjga7cYSAlPhX;;hK!5RzsRxO$PK;{BKjLNdR4?MPH$&q#xv89kN zW6iF&Qq@z+f9}W1PjbK2Ds1KT55n0PY4Bt}evg2Q_Y1uN%N7Sw{wp(wYQiqm9t69% z=Algg*EWe)BA4&1T<*5P4xoy#`^mweT9g5?sqNk3VZm4BqraVjI8XW>%gqdmCV(ZU z$LCCL1?9d;mcCx&EoYfci@~$?0ZU5O7kDep&9OFSs}1K9Knnm9IQ&ht0he~u>CEdk zuv5PjY(r|PGfcy<25A6YEgehz3Kjv{$=c|6^o$_SM(t(vlYG1QVw9=hf{yd2=;dbk z)GOUCzgwJgq=W19^u0mHJ2~$a$xNP4;X0gvAXi z++(Ca(o|!AKYkelPkHZDT-C9Glmoa3aRMC(3T_W#2vg7|P#Op`Rz?0yZs#oNvW@oh z#J3{YdF~t;eDoYAp?~?zX%}oarun(~bfw{^l5_M@!E?aaoZ7|#{?Qgv$?0!jB%x{( zkf7LoHwEzVBABO|g9+H>=>sb9;uxW)gCkWccsqs0;uF|Y|4G$n}l7C)B4DT|HnNhu4s_nGEv(d65B z%%Sb2U;q--!y4b#V$c-CuF+nJE{|^zHMyPjJfl!4*scOLAU{5PgY-tB$+M1HQQGT$ zdgyp4XLP@s<5X`V)DbXI=gcHr0fiXq&XC}54Dod!w%pczt7?KKAaqop#)gCaPJ02& z`~Hs2cE_jdf$q}Q2KrUnv#@d1H%7_JLi+tH9&whrV(N__Y>Y%6zP_uHO3xIfm?=Kk zmBJ_x^wxPPhE?fLQHc36>E-g&p>5c5b}kO`B1it}11dXLM@kkbPkbG(`u2LGp`+zK z$a#b?Tq}Fv8&Iv%+u#duftsnZqP1}z)$--%6~zJxzT%m0M%rcmR!#jwumI1u@D)nD z2>I;Bf2PG7jsPhgHS7V#7Y`yw*(u`%uE05{8w%Gabq5?E2;!*Y{f!q`O9j7Fb_n6w zFC}uFBFxN=xJr%ohoa|lVzS?Xsv4!KR~$u%+aG=XArkp}(M9DQeaDb?d4%jaFCVr2=uD?s} zk(Kag3;ViGk6x|^TV~CDx6DrllzU_MDlR?WDRRr~ho3rVCgb&|HouI{CPBj1I>6{E z5wUsb2O5WVPV`j7l%RSfj?~ZBi9&T+O|qk+nRewin1^t;r#JGcBVx`w7>@6e5t*Xb z*niXL-vq7?3H+q)0_f&=P==QQ53+#H1z1A=RgZi5K}dY-P(^)aQ{z>>+pEMp=Vs7v zmZ9{STH${&r$b5b-5TXp|xisIe0@lto_RB8f%}@*P{GhJxRUyO0 zy=~_J)a6uP?BVxlC-RHLJSwqy<$*W`?LGdl4JNYF(N*a#F2tuD*Y8jgY1H%d8c?op z|Iisk*SX@Jj6`Rp>!rTdv5GEKjhFXs!6D7afLh^nU%NrgNY{Wf%D7_!;=}SmAW(Cg zXjj4CF)uqPgxGfl7k~)-0o3uE2ZW~|=O_Lw8>p~EKYlZ8o0(k!_)+Wln$79ina30Z zx{dQELXqunHI4w+!pOr8pRDvguUTb;%Fc%NxR<_reevbjOZGEgq1|~m4!SVOzE>U( zFFn{x`(is^^K@hW?d6{lxszpD`p$ucMGkUD7daMwp7mILhV`ka)Ay4Y{c#__!&A&t z5rm!}bpkav*T7b#-sUYbaTz)ImRg*3n}&HKh!f05;JN^BCIlV+oA)Fo;zT!oUu~ z$F|^td55#S+P5}*vq{gSkuTeI3ssH7*-+*OaZ%zi6D48RGtYk-W!}5@U|Uz##Snh! z%krN6g-ey#kJ;*Oqa6y$3t4A*0c=yJGl5}im1*1tc~56y;r=_R_)?GusCPW~$oE_{ zvg<7KFjA{CSa$8YY<%&aFr3yH= zr}2WR!cujo^rc*xvXpX&WnhO`hRd(mf%L(V4J8h@-6X)r)4+F^NgrAOM_OKm@|+{* zK~K#{7PtVOY*^2=Dew$=cVmF<>`ldQu^Rds4yG-9FMB;59O0{nc+kgT7vlNM8#aP9 zGNsWuJXc+YSN+XnCbAyBu4Kw$Mp)6sVtA=Zv6YfT#8XB#3QeY9dY>KOGqppcw@ zB-Qh-?0mjOTN=&Auq@rT8H1q7Ysb;9#TX!C6YuN-qDox{rf~XhwFtPe4;Z%`Ow}D= zO%}kPUoH_wefeLlYLVsCiIAxVTkUbmnGXrVXfkiqskS_d}M##j%qmQrX zb3>40eVJ;<+qWcbeC1Zj?PhyHEq@#6bB?Jdyh%L-hIs_6ui?EhQ0W0I zCU=?c5fh%* zdgHc%;fg`SMQvT^_16$l^3%VPx2P2d zZhT(Sb3T&Pd}2zWAD=e1)6*AvIjbbEJpk5bYB4til@rUpaK{ayo>DL^7V7GL*YysI$nZK#izaj+HpEo zkq9K<0Gx4iu`KZ){Rf~O1{9AR)h0MuB0xXG4Ji>NFCK^zsy{tqs$_dRJ-mV@Wk_vr zQY%qi4>FZ-df$^Oz3o@B$P#`+f!Xx?1{(t`RYg#k-W+p|V?tuptGB5XMz9r&p6|?P z&c=*2m3s`(c8x;5cO;bolJ_N@F9%T8H4vcUCuhhD(FYEZ`8HQT9L7_8LHbY*Z@$BU zkK)u<;Lb8oYic6lY##V51=J>?%#{nI08qM`T=p^(u#pnrBR2FZAeI36mq>Q|#0r|o zzGNGoU)eSzqTr3$=C?3)wM2Fl*A7cLWbLdO-^2?`&(;8l?UoT;myFk<>*cMl^*4D= z^{V-8z!fBOPCKy`$~rbvMH*ctp0*do}I=V!>y!&5O@u-@t5&kH^uR zJ)n=e^a!J&AZIzfgf-}TswL4gKe*&jEjQq|Ir@9lvpidd?Tg=@wof;Fm5@FE-KiO| zqB@pmULsEN++uFEZAyS`Snb0B;HKnG8Mn%2xOaH zcOHZv7fmo<%Ux<-J5KDr)&3%#_}Rs+`{oauw)EFkg-*R!jf(9L*jH*5NaF|2SKU@U z7{CAC+F1axlzGB2fT;||0*Wplvpo)VG_vE)dVn=KMqd*hrm6z31G(G()cFMelUt7- zfIj2;!`v}{0>eQ0AQSl?+(t;ROTo{-VgQu?+#x(>Hz};nXzwR?fRdX2y|q7yxwW72 zRO&BCLO@I|*MQ%G+QHVg_FsnbjnP7>TWc49ohFI*W7OgqTl)zPPU%VS;ycqovt$>GA+PVndhjay>_Nnat40R9Oa z(T0z&_4sKEZ}9fG4D#wnfy|?yK?IOtcsGL0PhM7rY*=N5z)2bnLgBNFlL7Aj)+J9D z9;+;brCv0?%wl}~wgc*riEr37+!2Q#1=-(m5DJ)rZe>tNUI9(K%)hY89s{?(AROt$ zya=4s1}MEdJ-o~}`+&L)xcx`RzYerweZ>u0THe-@zB~4Odc@V7US2N0$o?8^Ph#HvhN@tDsC#W+_FVg# zmH(CzF^p9u2H;owlfKX{74UB=X>`t!{SbqWwOvbCy^NI*S-Lzx%;AP*b37hug5uuk(UFp#Wr@ zPISA#^uH>CyC4Ab+LfL0UuDpdMqmB3{;MOPVFvtF)UU>}uSZ3QU9s3Abea3|V?ovj zFP}Y&HC?yv$o-v9lhOn)M!r1UDEH|6x^;C6%e=7N>QbUC$98gA1e#yI_i6_1)biax zitkx&PVQu`&&s~yw^>rk z<_>+OHx}Qv$(}w>zbEVVJ;f?n=)=dt1*CI!ZQ0JabNbd&b^T+RV80gQyCc_QH+#{% z`d`0q`BWCG_T~WeRKd$-Dp||%l)qxCNe|U6;JOxYvpQw^fCN=iH?ibUqhFsuZ8{^f zk$$yGs_4ePS5r;zcuz5|IdN&@NS~VB53NMzEaahCuqCY91wA`(U7#L8Vm*(Lf?hbr zzDt;~60F}YrgoU`^a2zUZ|ten&V$=B((Osh~T(hMn*Ax1yzbza>mT((+Y0F&1!xv zjT&=l)IQ2^F_$K516Z23F1%vQrD=keW|`;10%PF=b9X#;n|gkk_bC1MXKJY(6b)1$ zm23E1{!KWNYnX6iX-!1^$G_0^r$T9{UWd6tSzv|w_sR`32dS0@R_IoExg33^*Rk{; z6&(K_aYf38KAJBQUv5*6;G~OAC?5{S|5Bb4KJzi@MVSW}BL)H*+)}ur@v@pUSO!km zWQ_ZAF?f)5*zG9h_bpXoNkrv-ZI^n%ySMQXz6yxC z=9j45D;WBTTYA6;W8D8s4vSNphy6%6Q)Tu<0MVvN{v6X}P83YcSVpbo8GSKS(2#ys z8Y3*=cWNKueYnYx)&e;N&hp$^j6WZY?-y{BVkr8Ca$R6!v*C$Xm}`=z3vA52zCW8O zf!SR^0;p%alJt>$f=tY+vo5EaR2kyWToRRt4MwX+u2{e$?l~W0JW54>OR0M2^}&+{ zjB9Q@W&2bXF)wjzXJe zW|+Ka{rKr==4dwDKFql5|7YiOc@MJl9aH_bzkb4yWZsoItzd>KOdMM$Gg*GjWRnI% za^ZJ?rd*+JJi%PdRC_S?a-nORF42F_*m3JLE!tTHxS0Qd9jReR4_m%Cgbbk6mxt45 zB&3~8o+&Ojw)M1cXXu^E-4AmN*;3|Al2sW)-hR{=s7$_U(FNxBTS%aNWQQWAGHyQ# zR47MN%W(%?=WcI=yz+(k>^;cc^)hECW@aGSnp?o_p6*?9E``kHHy-jh$$WC^DX>tN z(!Q;YBy|P|(NXyOy0zbTgN3`g=QP6^s-1^CL%?Ho3rdhx6>0klc$(MyH91W92I?Wy zukFDu9*|l4cMZrXw|(H1X8hsAG05{+cWFIm%IPe){qb$j570Fv@D1Icxd8nH=Gxf& zpD`-O@|*C!$)cNZtGTs>`35{5-R%o42(7vO1jy}kpQu@L`$fR1z2-nSu36S$U|Fgk zPhMkK7JXr;^MxmrkMm+PPblMVGJ~h!mqn&Oo;v5WW-0L?4tAyoN;xq8aO2umV75)~ z5}BdvOE45t_MAG$aHzU;(GG&O{y#?N*9GH!*nj!My-ZiWcO&*1(?#a)bcxB$suIh^DNdh?(!jq9h#^8NZiV>yD!(F$acMSTO7mg_KZT2jlyTO5f zI;`=VS31JoDQiubZFV&cFhvS<6M3BQq5N7*9W>C6yWbuN)H`m%g;#xo1rlK#x33 zKW%$Pm&G;+Q1X_Z=)UqFvHw&$JmOqf(rWroV}bHlCjxF>-GgId4KsUQPykh~Yl( zs(1ix?l7Es$WU&<6QbaI!OL+YhIHmxBsaI;ZqW_+#+AJXZO}Cmq%%S;T#H(QU(^D7 z(O%{gs~^`}On1E>{Ca_Zzo(9K03osFXJYOiJp=oP7Usj}na}zwa0nXe8y7M=G1ykX zPo$E9qy>#55dmEh;?ml!j$H=Yhu6TmAOPzuFNM}rk_)Kh^}E?@^jm?)61+;wMxSR= zT=v?7TuORx#FyUQc*p}$)X#iZBz)d)<~QTaaiU?L+u^LpRz5R0 zK7FpQ0HaNNHMwuU6#3a)t16E=!r5;yc#y|hD^nzN?a>=TM7ATm_!K+?=SWx@znR}m zk%bx8y`)Fkl=GUGU&rFO<5tLc3Rz!672kbgOq-hOI!$2r z6ESmF=PyL$pho(PxZKvVmm7KjnAxaeiq@Pc1Q_0a`_4o6V@GvUy{Sj|RB4a!>K}cb z&HaEkmCt8J=wDO)H$Kd4e`d5tAiA2QK6A@Fk@+l!t7OV_xX2j>9}J;B3$QrG5B}*X$BDe^V9x(XGiYKd$2Y)TGqhzzx}@| zejE2H4GT-3lgizL?Houc{C_|64LyQ4>PXV5x1*UR_iwEleI9RjUEX7!T$J)}4|nYe zsq8FUhkNh7RbR8je_LL!wNsg{`9+EIRgbElYmVaIA2sXK!;k-l)}n9JTmIJy@+Xl6 z{@egGZcC#t{=kBvBud^x=@-2CcWoG7-hi8x-nnkBiR{b{%sxq_j z`C!>Nl`F+>sr#%{tiD9;*ug(9;e=73VRKf4U8ZP#L-K%h&K`jnBH^=?7w|$n)uMTo zJ1geb5c7W>a1r-2%SgHEN&>?v#f1%zcG$0WoRItTlb%I>$R6n^7;CF0m0H2gLCECc zD>3kvHr#RH3S7;JnYI1bhoqem6v8(Mg*7Oyd`c77(l@I*a$NY&>aU`;;n7$}!DNYl z9jUH9f-OcEmau61-`sbDTEvImfu*HaC#X;2#zgZ5cEc255^zae8%n1mjKELy35G!|Bdd#@CXIGzGNZmhD-gIe#AGuk^Hmp zCJsUq^jSQ<_1%kKe*GV^Qh7t4;9tOE)VuzC$zOrd8y_8>?#3+J z7~TJoyVyf0U(@VGp7T}D>~A?60^TI;U1Qn)G5Eg%kMBGh{UKf-_wp~kV}EusPmf(b zoR`9$`WtBdpS$VTnBDaaG7wv(>F^8t{L{KsL|9aJk^n}2(aHMf5xT!nc~a>s-NGav zj)h~+|F<_&XWp#3bK8G&vt;_s)PwN}LE?YDWcyLW)Qc>DZ)iO8W#6AKagU(irA$o+ zAbogyuZ)O--cLOi^fm2>|3(3Lw(M5_(VXRV!3xQ<5aV>UhBU9Y3L zbr|>&sU60D)>^^K(JD_i1jGj8-9AfRc;+`-R^x_I(r*hteeR#P{I$}L*dX<*@|b>v zSF+FMt&c)Ji*O6+6`=g^8}usam#@KGsM+k3btsBrE@yjOo&yRCU=i6Mc{)lZje@xrpNK7khM7_gcZ@l?#@Fvzr3;7W-ij$k8;gbaz zs#KEue5dXAYrQbz>Hgtr4u$EsCAU@f&Y%h$Ds9I9`D5F>>U%Ls5&+WJ@$}GJ5EdXsBhBvc-?#FbO${ z+C*MebEN6Gn?<*EJSRqx;Qqj|rO7^Y#*2L_uf(FKwzwB@KymTgh@su_NBBg_FWIwC$sa*zkqLsWE~5f5kEWmYn>~>%4f$6k z{cwvLI1^_tv(-LAD=SuqJ9&AC2en%o>(G2kkyDwZ(q)fKZeJ~EEj>}GAaW70*r8t& z!k4G?a*+J#f-*OU^28_iZIbbdSw25@B!B+a+Ut)Igf*CY4oGDeEUiqw*zehMOc7rT zn-|b2KoBBeT9KzS$=**o{%D*2?H z0F1Cz&bD-=iLWMc;FO>I(|B{L4yggg&q;{d&&R!|;>U+Megp+)k!f1Y>n)%Ulb0%+ z+-)v1;BH#_qq<Kee4^?79tDaM#GFaOInse$S?s^zy?!HA-mFH&}uB8Dt+T5fLLP%-p)1k(* zDt_GNtQRdPf<}L|;7?m1=NPXe9?l8a;8)WIZ+hSEB=I@Dvok|OVT&thD@aGW~9&9 z_QaE49`TUa@_#k+Tvo>}6j~6cA z65&{fu*GvG%ka@yJ$6E?g0uI;3v`G%_2>BD=8`^H7oUM+xh(D&rTdoHg8JtcQ#r-# zEjFy30mP=c8i9UAmzgigaBR1&YaQvci~@Jb@&>pk+*h>Tx$^7SVEeFe$VMI$SwmO< z;yIJ~+Phr?r#nq{m?f@^+`SGfHR|#&@y~c4F~gDE9<5O+oL%(2+#)s>w!20xPnH*;}9oXu%(itLp|B|IIL3G#|RX1ME zCV_hF5#uDRZJ;1IOH;0w#rjU=wZ|sJ8U6I1Jf_v%h$rtvvxN%yT=qO#+u`Hc@00hg zdaD2{0RxjZ6h~b0{`p3@-C%WIpQhkqW->V$(W9_jE46DpxVpc}D@a#I!SzQ^X#vq} zb5N}EyGx`_G?LA4wx~T7qf8l$3Lz;Oxj*p=E*no#sC@Prms>H*I&SJ>lTRB;b*o~Q zntzW4C!Rw9Fsmu6p>OFSR#E)a-hyD9iK{wkl14eo-tl?suyNy?hWer9io?sFN8?PD z#VY5FYwYu?;>L_6s;2L`koT8V4eAdc5?D5nC8Zu}Ck!1FmCW2{o-;Pp9u|!-+OH!@ z=(i;L$M|g$-~f&w%H?a_c*smi?~z)H`)3JUe@Zzno(cQJWOf=Xw z(&!sHFfeN-sm%Raz?Ib5?$hr<+&?&Xh~x9vGbfDf45e^R5!LNF?pO^%GtsIi!ll(P zK1h|@(EUqUcj8no`k2X`$&KB;g+CNUQM7Ycz0uWMMcLtTGkxwRTq1S_M*UVCG5G6139ZfyLaq z47qWyl2vorrP7-A;rFj8b2X7##L!$De$JGDpFS4&x_C_(){!_EIVV zg#tKbx+c&vvO@x{*@={|9~R8;=b8Qb2GtMDwF2LvDp_tPu z;}#Y?Ac;*@I=z6UQT0jHaeNtJX_a41T8Zd>~u9)hrx?xBpeEy-8 zDsT<=*u?hSd|8E_E9G74v5lYmai8 zPSQ{8ySYumXw6unqugBg^~SB@0JfYhrH0tITgIE-S^Q8WImyS?fkjguSwvWMERo7- zI>eAw&Mv8bu5M_y zSdYGbcc8RNk8ODP7_8HjeE$_uEkD0KYY-6~OA)x3lD%F318|bo?XLdh`LIn&atD+^ zXa2uZDQVXroTtk+Y~1puA<5-4dxO47(y%RztiJN?TZD9UHJ;*{eWPYvJUnm38 z*aATR9a3#W=N{7f`^_<#LuURhUTtneFdnnHVC{XLU0!O31f&CeV$UKGmR=0+`B zRUJAPh8aF4CyaDgdwzmSFrVcl`&$zWli`C4mW@r*6|=>TzKYZs^ZnEjXa7SK$~fLN zZ8eWCmu&HTBr5oG1@8ShPubdo3w+UB6~m&r3n-1;D#zl1zhe=!s`a8@I89`& zgw2!q*JOpYitfN{?>=uWnrwBi3p|A?YI`EDQ|iwXOgRL{!Q9Vjl;pSy443l`L{=E z>9t4Y!7V5}#a;5*s1@PrXWiqh6}U40m80LiwQ2iMTwd+f^LYXYt8xLF+{`z(nw?>2 z;o9pHF!Z3&`i9}w5^ZIL%5f!)^oCaD@oz58oX-3tN_1H6^YWAaQV#juD|QEblu;l$ zl`MQMet!annpY+3n|XAnHk5IklDD$ODG@o5+RXBwyY$Qu*bFc?4xvOKGI5mvA? zRF}hDkMO0`x^Khx;Ant@014%Tipi;oG*m+0_5>-)d_0XWz@<5&+u~0YNSYBdE8-pP zz@f;kE+bzzXG+>x*O3m&EkZfPGnnCrBV*3Px*`SeTVmU>uA%+aHBpLB$I!zE#ie}N zGfxDRF0bg->sq0JYeLe7G6zR=QIh?3EfXnc6y=W3@6vyc7z`e#WAMtkAvQZcx0cgh z<;UPA`<2l@w3I`>x`J2K%l}9+eI3Fg(layUx}y0Tp12E@3A4j`x=s%IbK(1 zpTF_ZP~gEOyK~e(q9txRBU#G_6+U9>3S+p2>4Zb`#UGekD(Avp5L7>uV5t=@-y}m{ z#i#f-$yqcW!6r`2)QlO!m9-8#mwps#d@HJujzswMidVN|6i>fSEWlJAHGQbyS8GQ( zVJ_=GW|$cyuqvCI)K6>5doUk^9N4K18y=ZXJ5g|$cwHYZ@mT?v*nSTWgjZkqV`X`q zrsOi?;yGV8$z$T~S$@)oT0RV#9Y#wS;?+`qr!HUPR%M* z4_??O`?`v^%Fz2oNl4Wa<&uG2`t@kt75!pNYXqX!Wn$1=i$*%25J%(S6c5L=@|K%I zS%YIqzX+UkPCyG*KD&}teI$OVVJV`=9yAFmUm0xE$W4lWfW$lp_ zp&l^ua$#A7w4tz}IyKg6aWSVF_!=Q4&yg#eLltH#9WR4}NB!*#te&X0TjuPp=e%w8 zDbe5=nw+BWN_3EO`PL6du@Q6YQO*!=&I8IXWRwF>QC-h3JvX#~?;vNJ{>gqfl$)ns zG*e3S(9hmOYe!N~)OMT?K@B~^cVfSoze@Dyj;WXQNy>S)Q{Ng*EjMoKzz(M=c=h{N zEPP9gR8-EcY{EjOq);OYM{qw54F7d0AJI{$GB1#4Jw!0OHQQ) z71{B=_8N13k{dN`a+dQqM+)Rk<=mPf^MJ#j%hi~U-)c&NA*TP52ev%GjVwJb^ z*lal<8izr8ed>(dnw3-R>-!ooTi!V9hU8)n8V-3M*IhYfisLYLEORt)>L!MkR9d3~ z_jyal>1`bj>#|+_WBujJDoYU{Km0K>HHV@U6|*mnhJD~vyoI#tAbN%Nd-cjxNu#%t z5c*G|1deEoVSaE4%*a<&llr4s$QNSHlBchOMmIC`qF!&X$9FRcQ#ob(JVfs{EQ&JZ z(o&?qO<8fo6ljWX7Is`ee0gk3Ly$RbjK_{``RtrOvG=r?w9~_P#1wKSGTFcNP2;5> zG*e0ci}CB73H}Y$BvKRq;>u?!olptK3ZIqN-y|*TgEEmR`%snrJk|Nl$6@`|B=~eb z$50aiAGMy_#O1}p7Z-pf-!6_PyV?1+7*l4x7Bog=R#d?3u290A15u&5; zVUlijv_@^_o}dwtb@3Tl6Ah~gAZdR|a!$Up^uYs9Aw7~YG}GkIDRWKKu<%)pDRJ121qUjai^^()@5H|9#ZT zPb8~?Ly3lg_h5+@VgCc2JxbGa39w{0;B?Lr%A}e*IlUBB#}u-H@y3qJqwgk#%s$sI)`b zV*T>*>QFsX@qNCZzD~YW=-%x5^+jpv*FHB)40j_mqFUvtPG%d}-+VmC$3WXMR7N3Js>$8hOLG@(9ksjN^mF6#z+-4IwgO&p1^UvOm^;no(eZ_Cy-9$WxX+dLn zGLi5m{NOm6grqbKKE5BrWue@MKByUx3+O0BDJ6fZ&CT*Xr5C%jv0>1P zJ}Vy;+Exh?vZA$ph@~dP5)4MZlHY}i9TcFE{8sSYwq{SoZWMN%@)?tGEbz}KHH}Tm zooh%&>QYnGz9auw5nKdi4}HveMp-%kn6EQ-jU!$@gLL~|XYo}Po6H@H0c5+~`$d81 z3y%WCeF|?a87LC+8fMWd zH?A3m2{mFbzws#%js++QkkR{&mrcAqV_Scd*yTm334-6D4QaHpbz)t#KILid-Zh?? z;yLtTkqM3nZA7VK;Za!_D^ap7kD2#Juj<}`m8Fp>r6V4Qp@#@7fnbP*$u1kSjmA8m ze7od&eI=nz#_U=D_C1jpue{}eAvyi|&AfTR5i*7{m2nP)ywaaKCac0In`@t_0BD|k zcTh)B0*0!Ils(bsPu!1pSy`En_uK~Y@#V0-rJ+Vcyk8$}NRKbrz(v43?D%}HOKW%h zEAnzI{N}VCdG10xpP(xjbt1lM-i=BLv9}I&{6pzE{5Diyd^`VhxpLYoui~VMZ+Ea= zc830s{@Z}rOa#QUuvpQzwn0FobD}97G!8hYbpHsOdVqWHg1;CwQuT+FZ|wCw*q`1; z$L@Rt?+R!TZ+FW_rDQWt8t#ZZp|CK*X{=4s|IWo3f{7|3( zTc-f60h^(a8_G|y_g;8^VUvIHKsjH~ey=XaFYjLj*ScQE+2^9%c;^_*|=( z*iO%GpS;33C4MvxoCB*o6tL9ugOC{Hh_2LKlJ4gy+Pg6LSY{ao+?jV1ZzfWpUc9*W zLpOT0-?`e=9Cpwj24el5&?&T^#=OrV@zwzkqT@L+y(rPw$hnsGb7FH*c`t(Tp#r`R z2B&>A?csH}#$>NmWy+`o;u2eN$-wQ3ca8O{uHm8>r02@RNdfunswLk>qGjjEBP#0V z8E<$C0BwZnkXjeeDx0-{o~tGdMr&YnsHBnH!c@iFTvUCg^8Q3k^WK?v)?ll|Lw8v2 zRd>hvs_;C6R{`I?h6W*ySVqZQoX5t?hIro}2DLP;C>$*|k6kfC+I$a4mTw9)?DxBB zFt{r4pu!H}>OuC7v1->UW-Y+*Bc!WSW8oa~b93)jF2J7fqi2XEA*7&tG|D(^@)5oQ zSM2Ny_mzRQaZ>)!7;t0vR?w0V|3qJ)HO9bLB;5IQCcj&`zPXR(PP|(A-B0?QeZrJU z3^ARpexLt{Otk{G?AD5-?83YCZGDsBsJ+Y8xYAFTOdO7&Y~nD<181dkA~QRd>m}Js zP6VhCx83BkO^O@&o~e9g={uGxpM;wV}~x; zE~Luab7p4kEP$l}xhmQYOP9}Au$tVPjE*VU-m-vyj=_~(pxso181mzl6Mfr}4tohh zs?+V-e_E5fD&iZU%JDB#yES}3+`QU6E*;b`~Jp>xAo9q0YN59o8so~n&C$=46?AUK=* zI_?3DtJ7)j9t42&@i)YjmB+ejHYpFd3W28m|B&~cVNIr6yJK%4 zSdb1XA}Z1ZL8Pe+APS<=M4EsakSe`n0~9F&(whhb>AjbzNN7TU(5n!74>dqY&hrLz z95?LkeBU_xT<83l>jFoUx2$rnd);e2%92it{Q)~1C{aZdZOml1RtCkmoIx>STr0?H z=Q}B+Ny9uyKNwcjdu6dTN)=8rjpRt%9$%W ze;*QKcsZbh#T#T)`x5{SZ4*oMOKJ+o-@7{D$K~$jO0<)ION>q|!Ws4ZDU(YvV(0D} zo6uu3aA7Y5xTXb8_g2T_hT09BBds;?B|i8$ScKVw+jm1l*TRU zk_y@bj+?x#9X@*u&UB+h77SWUM?El7y!IdfGYjcLT>-0HXDn|4jFMU?k6WeRp?Gls z6+~9^0qOj(*sjwlqHW2$QHf%Qli`$u&sZ<+SSydtz~%0m2U_-OFpUm3>Ih8+-78{U%e zOx>!Ly%57V$?*D>7ke)A!-nRMpr8rO-bbvUzBah02m?c_K0eOmm2&Hy(`O#GXSjED zV_D6j74au2L|+}c2AX9LA!J=4dL@TP8ec?@5SkilyU4uP#y6+UnAnL{yfqMB;N(1K z9o2&^BZk-XzhEQq^i>ja*jSwyQfKR{;Kq*Fst9M_Yo5;aC3fRElNYxnL<|C;+1RXR zA^#j>jmILK!z>r;FFb`Hz%Iz>4QsE=qzx?%OgeGd^i8x?R0^2Mf*DQ&^N^&RJ@kgT ztrze{g~RzKADm$nv((Chn!{U|8E#pKI!@JJ<$@=K5ht=$chrQa*khzY0ofg00I{Cj zCY|n)801Sr(@6KQ(ut7<6Pjj5GCP-y9D;Xk*%94zy*={<1ht&@G}6I?=ap0K^aoSX|F zx*e};+2e(gT*)$-Qy-0A8t>46sDg9-s%5T)!|z=!*LltEH@``f9tyG}5&QW&tivvP z>$Ao(ak=X(^|2u-ZX2)+Bu(bQ>fg)r?6BbOtBz+@<6v;SMHG2Nn?%!nrMS2en|Y?2 zcnB#s>L1bc8SpYQ0})OH6I^7Ob1Ai^Y3-$R54A#geM&VP7psWX0s#{#h%Ohc3|Yjv zcW(#;u}=EqviU+bmxid(X2pf$hlWRH&}yMVulhe=o?NsW%hH+fC3Na|67J*PyCp;$ zPcIyp?;3qHHH^iE*EmsYg^Pck^Sp0$VtXX3uEVGuxsS4d8Y+L-r#5Fk-2_ zm3ke-+1Ok9UI~xP`zdSC&gKcM=lai%$t}-M z4Y8n6DmCS?z~~a}+kzqF79+;!9ja`V&n~RitRUOgnxe~|@EP?8ynpq8__6%!W4SPzmiDMpSW|9)I#NM*q~1* z+5w@)SON$&^V&oTq2Bi<3!PSE=Lg8w|EC8*x&B_6Mzf>cZ{2S{T-!!BP1bq#*TZvWLhU!06p$kHQEVx)UgeP zs}*a))xA#!Huw~M5;CbcYm1aKaJkP5jX41f@Asa|PhMw(`;ve@bb3Vn;2rVxWO9Hg zxk*x9Y`VoG>YKX$U)i*obq~Oa?CxBGJA*e+87J?9D&*s!CHXgeO7jq{SXxlwB~4P9 zt-1?;Q}C?hpSpzK?pFO65EgKybV+RDUmafk+Kd0j2AxoZm}dG5tKv+s!=Shr=)3-= z4~=3w>F#B0n?w^4v@SIXF;O?Su1do%rL2zc{mT^&T_VW_g26BZ0Bo&5?eJp55A#|3 zuxBZ>>$D8OV;MS6`Y<0L9^SPs^A2q5dittA_&j~H!AQdp9qCg{NpnFsI2bhf+Z5r4 zPr640f~juc$VWGvIykoKQLH`s>!?s6N-6&usecen_4nN7S=-}$^d6GXp!xr09@UJ{ zwuk1j0Hku!-e}~bl5#bAx)kzn5IX2dUT$;k4~663{Zo zBPrw?*sUbV;=&>Aj1W5#c5Fl2U|6+{{uceU{RrP&eeH*QIg)L}e8kCoT#pNZGSKVO zAa0~^7xx|3_57Zq{1#ZY=1Y=3MU6BztczT0v9eBG?Qfu2KZlunLvB@QOTDe-`nrkY z!W#wBP5uU^A^wIuaPt46Px&P|=@>f_RN)?!YkK;>=h=V1?&0qV+R%E3gmxu&NPYEj zx~fP_7+0%V(rMKlR1K@O%hqQB6U+WK%KJl}xD83~m1(-qWbE0%$B&aROaqz}O~iq2XLeUNgd za2Se;tkWg4=L5@Zv|__@_NK5luPg3@lHAc%@bdCVGBA|`suHv)f{B&?=LTJo+n;4kr% zPP217i#sc}5|r8;zU->3>2OxfVlJly&>g!Xh#itz2I;2wNhc@$b4x8HXV^E&fwj;8 z4Zwu{&%?g%Ao(GVoCSkZ$|!iZB0G}V<`WjknU%?{x-9H)W-q?mUVjHjvF+Gn?R9ZW8mU)Gv$EsGvR>i6n**$j|( z=i!&8;CC&SQkXKWau>Sn>M`qaAS4?|+DSrR#u6=ge}t#+ZQwc^DXb9G6=@JaK>~LM zqcsmzhg5W`Qzu1tT$8798W^T_DLq6_y;ro49jcZxY+RsDq<(U~pabHAq<&2MLb27b zNu@N*xr~SfD%F~HvQT;}%XW*&)FxOHxj?7A!#U!UjO`8^0eHaeuRhS8^7F3iN7RuM zkO>HyD{9&E59Nm4(87~+BxjpDdS>Vyk2~4sI}MjAy8@Z{34{Da+a4hL+Z_QssDj{c;7j58Q;3JOS7$AFww^r4~2%Gh)FzBJUu#nP@vmkqjCoYWI5|(dCBCKn%;o`aJ=a`aH>?!A`M4dhWw(gL&_lkR4WrY5>4Fx}7X&I0(HzU+-=7v2`8;nUF zt_5>-V9XPzpQDW@Z(cbO*1nxS$*=n19lPFLuxAS#MKUc`2z8P!pal|JK@%U}c%M*e ztFEP{9OsvuxkrWPTcWg@J98NSrnhFSX(>x1M10a?MsI&yCTQB0A7yTbbyj}nOzqhI z9wwr1e}1e0b+Kn5hse+1@Hz5njk;l0W62Rsp;H}}?MXd>n7#C~t+QR9c_$0C!fgh^ zyQDlVBHwqIRUgn;LcLT!XDS}<{V1^JGF-^(CCmzGct7e8I%wFu8`j(Ez+)G+@8vN4 zE{o>D-d1%iU5#$3`ME2dW}eoTa)~ZNH+dM76|6$p!aMiH`b)Nw^$f>G7!CU<$#IfG zd3^fJ1~`F}^sqZt1zyTSqam*u*)7_Ww>qNe^p8u7?{N7V%Xc8{r=?-fV!8@s@;BS7 z?Gm-C79-+&rmK?o0EPdR788s`eTlKtve3tEgX&CR#X@On>!DFtEWAC+3+QMv>*ISs zXL1io?uROjL%jcJtbR^-MZwPad81}p#{CoUFe)zl!x= zmRhn;k1Y?xo@V>kP6hKzfoLc*FPi*Pr!+SU`qr8jO9D?CRpp#{NZY6U@JgROd=;-iP zI1$O9CTyT#cI&qgG^Wsgpi7J)t?Z8AK@7fy#D$OCHAy) zS`^(rwCC=!(-e}zu%J{79VOaxPsM5S%fhf1DJ)!v(4J&hdoI3!Y2|wsrMd+kg3A5L zmK68L>#6xKJh~7pEY3RsxK2;bm&?(m~+l`*U{twRwPam*U+oIj}@JLY1rWb>!u@-%$+HKxiCxuU7Ps=o?Q3C^Xf!DtO zltHN~rKv#j5-OzCY%}u9_u2gq90(T6Krz^aPDJEDFjjU|g|J%3DS2O@68Ll~*v$yO z`?+3G@$O!g6l-KF_)+^*_?%m8#0_j>#D$gneZ+z}w>ax2_V9=N+P6~z-C#w%Dvc;z zQsJ^vtbhOO#>*dUxa0r?hHhWmjbS_PjFknm)6Y2`|NdUKX32s4Q>F$F70R}hw?`qdaDb3Kpx@YV|+mUgLHQI*bze!?EHXEXu?U;yQV)(?~oeb^sPjH4qw}smuo&PAS z1Vsw4XkZ^fdno*eOIUl@k}@Q!Dsh$65PW>f=y`vBhU()-fl80mw5WSkj#x^lztEGY zrLtfNr@E-z;t{CyT>YYQugWIVQuXJ0rC$sut_io332&}`IlS{tPTEzh0n5bBv+8T6 z_Xq zLhR(;ygm-2p(SD1XfOdGev?M*k7;^EQgCyFzYU247m||lxNHkit zby%oJ;F=*Q&GFub=s^v{WH9aekqj&=gxbCS^pHzCrlO8p;&gw8GR{pvs8 zedZD^6>4c$gRvw5p}Oin-`S$#iaRNErgU3XyRYxWU{NY?VwI}zh>niTWQ^5&k5O9* zMaVzS(etc1%`7i)-UIKt-SJL2|Kei_OSBv$&ip^$-0Fspf5wg0;YbYC7v36w|Wr3Rrbf4 zU_0`u{qK5RZb``3RV#p}(5!s%FI#wk@=tgZh*@B*G7Ge!_#3;g< zxKU|_Y${7jVxZ??NBaCjDE>E|CcdlKLBru0f+qj#dgVXAhR6Fz8FG6FSl)5D=_N%Z z3Ut9urtmJ}Jt{!(r6?LfH+*CV8(3c?_O0(5$EY z5*90Qx2cQ;o?+A?&G8(@CJ_~y_FPZUGpydt+VaY=N_wd^bI{aq&)NVgbh46|9&U9W z(PC&b>3whoZrHQzDOt5oA?{{IL;|F8Nqt|?ai{sO`U9dmA3Qp3ifCqvp z5?0JI=;E^J`-CI9pZtmM%K#AI&#y)aI z2E^QV^w`9m`5%owWnLQFoxdNo`xX1dB^mbZ#%5N%t@pp|)76lX`mM-A-3>a7!>(c>XLoI09>gSO*-Pt%77fZLM-OSmKi!ptWcocl|U&x)eWu$f%%XYfK zLheYVs_-yg#PgeTgd1L9>^YwBTo0;ymUcSvm~TM4vmA$l8F}`fd5P79TM3t;0+32p zLO^~SxH-*jQ04Q-eWFqCtefQCIPy*tJJ%|oKagGG2n?!5HHsX&&+@LnhYl=_hp0df z0b8(W4+n2v%_jm$`Zz`tOuoj&UsH1R|7pk8F;MtKqJ)jQ;-eI5g!isV4i^+0~_Z?8|W&D5xx1;8fbmwSLy1(t1d(eVfNRq~HLG ze~Z_J8SV3%hYZJV3b#KOx+Dwgst-AFs#t>qV>|iT;VjR)U6M;P$ltMa+$Ckk0A=PL zE|-aoFdC`W_y8*C17J214jPo7{4X2@@S0#I>JV*I!nTNN+5HqZ} zrf&N6_m)Mp(jVlP+Ir(6@9z@7CwT_l*O=x$p?lHh+*4?3JHY^lhmF`5Delze~R zohzH^G@qZ8b=JDQRrkSY(gitH5lm}sgOa>$+eiiMb@=?&vFhiDg6!aWv=YjipD(NN zb$GVz0|!ZmLsKfsISU#(k{jNz-!ABeSwf1m@h8bgvh$$*<4l{wUthrLg(@8Xl94>& zSvc*6-LKA5Tv0=X$?42Ss9`hoqs<1%JA{4Phq?GFDY$T+iMP_;R%68QA4=1gC+x1+ z_uDTR*M!~|yw}H*Zm?*&K$*jM`4N}kcqx4K(s;wkWmm7sUUFDZ?tPa#u0Wr3Y3?v_ zc~#kAy@c2%SOaWRga>xMOaJq?gJmxK`Hfw8b7oV7g@sk0zXx`|p?kyx)73eCuWfkE z?wL2Al--@MF~iff;tN+(G(sxsq!`)M^Ub}>92Zj6>^iw_59Sgbca*EZ!r86{aQanP zb#qn)q`Z1cu@pS}Jeh&mAn#ShDpvevF0W(w+-Ca|5LX* z>6&4}^ucWpt^m`>5{(kwgR@CYXiuneUp=Jk$4duy^56z!aMc;tPwzb`5pX2{DM_#F z-mUfiLSFA+bA{bxwKwM4x%z7P?1^@{;gzBF_k=R{Ges0 zIwu|+e7!Wv+m~3!(kblAUS6i`qnT^+*d9~<)VFMTf%&kqx}Z{*;~)pHjkF|`$7Cq~ zh=JhLqnh^BO2ba zBG2r>`9$;8q=Qo?er1x2)r@hbXwTggkp(}52o^EzTg55EM9)8VnAT>RcDl9%-KUI6 zmS!>|rZtCazUJv(_U1b)!ZxTm#kjzo!}vR7-Oj(tx-UJPs`Yu@%Z?PtE4x`#t+NSn zPFGBD#(gGQKK5Groe!=RjlDuvT^B^bqLaQ;)04$QY(bbHSSY*ryx4W4Fz##cE-T{@ zn}vx^DP;FK>EOZ7{s-jF=T}eSJ`_W?q5oHH!;xupWPaof%ldKj!f%gnAA}zD44+LY zPN&%B37E(1xv8l+IS|xQywJsvH}!V1DukC9gtxDA4ya0w=#))nw)2D5>KTTsDl4-g z@+Q;n>a0AE+G{u?VPZ9ce~UVCbF=_EG;1j5DEn`mEF0n>V+zqXFU9HV*k6oJOREs^ zA4!_uOjsyWJ7MRj#cVU57u)~wB{H*ywQH+W$z0Br!J4ps+ySrGR^4H~c%xp!h07?W zkn-1YzV|*FVtmWC@9mIwv~A_`UB;RX);Ofho$DB;AtfT-znX~JfV)=yJ9njm+||s3 zA4%~dnVPHqZl4==Wtrj?yQ5`NLvwT0@+2xCD5LAO5qW2{Cc@CSOvA~+CG@OqbO^$r zGbcjLu3GlyiGz0u_`EveXE~A!=Uk&D-NkLuPob`WL9srC>T`N$DB>}pHdpo7x!gf;Ci|h!7j|MH&4Fy>}o-J$E zB_#mdtg6}X8z!kXO33PYb#w}SzZ`me2Kx3&7fkY#-(;y?QMtZ}J(^vAolo(J5}ix< zm0GuwT7`Y8S|@Bz9cHJvD1UFt-8Bk^d@sll4i|g_J9kWk_fz$7EuAcY`>iw)X!<8w ze;mp90{}FohQ61r8fne;fn)PMsIc%OFH_&PDHweA-ivY)1~HMk@mrJl$VB|!P`O6N z9nJoDeOBRu64MiSGq4?#zs4JHA4(O7}4C2NPs{%hLn zm$ZG{2+7#So%U=6^_4J>d zuBq{-5ULtam$b>t3qb(t_-oPRU6p4!q6)wGR|>eFM3(ozT2gV}=@M?!0J!9(Y?-H^ z4E%;n{-&Bt?ZjTD+inDD`PKAz`b%tO#fR5haX#|9Za<_6KBdi0CLy=&ZvPS8c(QP; z$?RvEbFL+5z8$kg*AtiQ1lEvCTwfY&eyx9SYfm--UWPT_ov^Z1Eu^Q+2(JmmoKHA zi#|xKZQ<(#D1~DWF3!}Hn-yd}g(pf3E^xP&-Qsofn^bIe2A_Hg6J`+M-Col(#aF#a zq~yqO&=I3+u5i6;Q?%q)QSNulgip(weL~&QwY~o+c;?7zAW9COp?})5aR4M0Vr=w= z?#y(C?p$d+T`z!t9(#jzrm4-}%-1+qW6NiIpCi%9K6oSqCU{gNqoI1Rg|E6yB;m$z zke`8LS&W|Jyo$RF>a@a)$Z^@3IXOFXTR2BB5zDt02#E{ZVSlDGzGF9Dd6AI6d@{}C zv0RBaGQeWf(tC0f4quSpcDMA%P~7t*zJs!UZ~R;$WA|tFa`P#EQlfDS-(6lQUaLSW zjOiH);xXCl8mqT=h<2|uitM4;`OLM9H#qBZ=hlW9r(+w0Zl$TWQ@Ot0?fis~pC2md z?rjoNytWq(-G5-c0u5Z^5j?wV_g{ML78XBK>YPz_x7y!v^!jm->F?1`9fw8WXWaR~M2DbcX!it$2hKL@nFx>XU9Vb{L` zxz#IrlTE_7WUc=#uX20p>K}bj6Ht}k2Q`4|Uc*0fb8P0znp1l7Jq`Mw=sUpP{S2lD zV15-p{_3p%`72f@C4cxDi^9q_q7UB$w53X2x#crSg+H3fmB-x)O_R)N7ICqyAm}IITlRsRr7skye!h4~5 z@fFEqRFqMR9v`MQWE=8G#JjIYG7U4xL(l)|cx;-<<9OM;$lCij$V4)^5|6l&$cuO2 zdfSu~e$B%^2abQ8^#WmOztO%gSM^^`ZYs{rR`mM?2sIQGvy zGI00eqU{C<4F;teJ;N!H_teeVZ~xSY`A?<9Pv*7%_k6u|42jFnw`~PWNbf=9H|teV zf3|RZf-n)EVbqn-$`8=@)cRM$g4Sv@1mzQM9Xfsodc+ZH2 z;071hU**Vto^1kIM!^ROAD&24FrK;f9&r!PnBIE7=w4G|(3v@c-Ms`k16EB!y>hgE z^LCr1AhtfR9nCllLs(`!3=6XfKdqjxaJ<{0;IwA(C2+)zemzURDPCSqgfjq-d(t#$ z8v8%&=MAPM8>Mjmo8T@1^~wa~%s`LHW>BuGUo7uRm*cXgX=Plu>C9^|BycTxu|HQA zvur9jJF4kV#qNJqUA;&qZ{gx_LtKWtXYw&I9b`hq42Hd?n@2AmA_lF>LOHk)cwDp>}oIqW|7Tc{oW*K{=Gt;9=-*@tR2fF8kOf)n#fzH`UX;U1#!Ti z+9$j{3Km}JcnMvqIXBMG>bSfq9h`d=RXrtFrS8jC2~Lk{IgijP+3VxY7IB0^L|E4G z>_l)=8Aix%d{KgSrifXB6IHQDZ6EtTZ`u^k=jRut|C?T_5~(^-+w;^-{K~eImv!Vx z?eFo`cB=VEgUXc{Dvli1lLvQKgdRGLyL`hUHe6)G93(g+D>&|cz-Lj@yr&;(wfl~iX|Q?; zFTLWGHX0on=@VEGIlkwR{RtM&^O^tiC;qc7ROj9=kugKlBlr1{-2(mILszh2cc(jj z`#go%a&17HfWExQO6-+Eu-MS6$O;TtR3B`A4|ilbL{I)MzFHttRG>Jc^?pbd2WIlM zO2+-lcW5u;f23_@buIW5yGCanU0jm&m#DlN-@HFA zVpmbPXr@2)S+GBASFO3>nXJWgBKjHa`89y%t&c0}uFC*?^Q608zG!6I64(ivpv(%p zioWvt!Jy!5VwjR&_(@rIKfqttW$>TR8a!QKsdt&vvp&2zb0H6O<_+A9Q_i3^B)C$K zLQLs^22{%RGSh{?3qYCPsPP^#A50(9-PB&`#kK4WP9D>2|J)y5frbLTp!O%_dv$Lm z>IQ_Y`qC8v;p98H%-g+u8zM&VQiDg~bgHZ2k{16-;L<7~|uX1)h^ZwnH0GMW{e zi2=)pc0<}Co8Uswa#927d@Qx(LeN=#_~~0y-geeWIgtPM0zRTT zqj`zJvF_IlfhQ<-3EReEZThfNB3EsdPALkZqV&cydB^dJd_{%+b@BacUP>BT=$>HTA@MBa4<;ml((I5(^;guK7LIDjdB zlB4C5HG6{6y=8lUYY0Y5FPOz~yEvM6OG79{Tpc7evzQC*rXlnC*S^^*bNniVL>jC7 zw@70k*CTbYpnloCl84F{AHjE*yNTDz?^D$~5e)bea&b9U-BK;x_~%L72Sun}`MJJ{ zJ7bM3spHl8!mw8k%}~qB;1f>UJ7BKs0Vm}^)G+yn)Q!oJJy7KE{npg_hufp-P>;*! zru16&{l`dh(?3f%{+rE_{pwsA(U22E9LfWu?cH1GG)?L5?yr4%#|`Mz(19iTx3gr=M>&O4ntcD*mD& zgZkp5;BbjADVY1+mRjC-3~`}kvhrGF1BYkHj!QgXdrR_l4{RLkz=aJ>|13%R3J?4f zvt-~dfMco!HSS^6RQb2CRJ}qucX*fi#o|}t7w^Z7MC;zB(lp_M!%U>(MUMn4$kK^8MMrpg<_h{CEx;14&!+x{F8}^ClNDVT1pu+jDD4_Yi_8~!1RN`l_ zqS~q(|0wa?RuQT+KbN<0`!nmgXMeU+*tB-Pp1*F)j$Bz`+Wwh_a0njo@ zG&nbLYGmpvyE8Y>hH)-uX3l<%rIkZvB;;ss;t z0yN;Bsct0p5)OBPH}3BLh~Bd;k{zl0^uS!dv(i(8ONT~yA3ie{qJbf^10ugi&i*s za7{P+4Rrm#)4csUMgze)APV@U#h=kS_3b}gCbdEeqhrnhKY9QqLZrz{sBAegy1Q1P zS!fi9G_z6(SiN}T@c!qspKl68=%D$S7ZN37U`pwWi~;b)Rv6D2k&Nrr3xRyq&0^zq z!$H})gfjFRmcVz@hi=+16pNBVv5c~jyO$$4BF+;y63)rY2us3_5;@lge@9)f!bMQa zu8d!@XB81oyxA#Omw6sht09Z3byIK>k|WGO*Fbh;p*^IdQrv?e(~<*@nb_lboaP-yw&4Ac~fwoE&L-* z!$n;>1pt`M?MdJ2#yXJZsggJ3YF$7 zF1X$+l|nhU@GkQY#joHW;Mns#wzg^8R*rtX@Fl~Nij6P)1nI)dK!aG;cHT$zZ9*ya z?rn>UQ~{M%2(u3l=rKRebJZNn6ESRPHT|-wWKTAJc+dRe2Gkr z;p8F-RAiHLeB>$8XJNmi9?X8U=a4NK>$Ovwo%P$ZV8KV81ld)=+L_ABX8lnEp7_RAD{o7oDb;52iTV3lB|!6Flkf7K~>H6ifyk zLy0bA;>V^Fl-p^<3|cOpneOB5i*${xNI|Nj?{|Iex&ei)nJHmgw;rjV0txgSN>lG& z-q82>$TMC}s??#CIXjx`_&WGSVUlvpS*e=h=+x!Rg|VSljkKwV6ex^{rC7{Oy?p4h zVH|0Kx^ZFBe`X^LnfBmHFD_RIMwOKp+L-mXU_4(jT6NfX&|^@-7&Y2-;$zd%XAjTJ z72CCcF0h&uJwrfB*9*h4L#^xUR!8fA@n@vaG?D4iDS16Wxr8_Fc3H3uc4>|d=6wr0 zYpYf457)B9im=-*RXQ)94|Zus4t^dQjyg22uS;hcF(1w-($0Q}Fhdf0f|fIrBge+? z;tdNl$}1EAV9l!`TQEjS+X*%!L!)XF#iNcdj8SwVH^C?g^0ol-r#o>_oSdgB2|glQ zGIC?QwKn7Shbav?^i;5dndEGuB-iKWdcMz`#V`pu%{&uLDYQvDE|gZpq2Z$=xopV< zx+^0JW>@;rsd!DW=}PSCA=^-?n!XQ`>ceiH*y{PEVNcb2i_`ZigmYT&3oWKNEco85@|W=HK8g~ac@568 zh`7m+84=2B>n=Pu+ui4L)oDn^r(I;es1vamiMN5vT0UaJjbZMw$?%x z(VIqhuq7JC74+X9BJ=gYVp%aEWCrfUrDn!y;*)RXxL%H)Ps+FQBuprr>xbXv=3WGK#fO*XG`Q{dajim}l-mSe+B!5QiJA z;_$$HZTulsgOi#auva0fqvQPTvjGnCT>|+}@YA|c%ZAgRe4YG)c)MjB#jx3S(J(LB zyOj+UXFwCBLTAg4iDP0#V*wQ_7&+ma1JLByEogEKjG0)FRhZnDMw{G@et3LVMw};6 zr}i&LWZkt<`j$!hn!OtaVIfiw=7;km3#5~53oxESRVYrxZ31?L-E_hxHfM3J$zi!7 ztGVNt^QDUes0jKMl>KnCv1(jG!pr1SVyhFYgEQh~ln}Vkx$MGwHP{7x)Ppw?M(3u* z_!O=+_io-`NG0ixbxN$BwQRf~i>*==^?<$}-F@orjJL5VYPl&moS97w z=FZ(x%a?mhF^p0K(6`g2U$I+XyR+|x*zG!LZYj1AAk2b_1s7)07-58?Od(%HYY3_C2+UaELN z?kjZ{Y`tvav*9D7C2ea!pIDufIOZXX?gYKi^hXh=*mlV82X9YC?-UvOC{TYJZ~IHC zMxa*t8LcX)r%0UPm#Z~r$XKlO9vr({#rKz{PJVLG{VPfM%f<5~R)~E}Mh>`6)%0{3Qn~#l8=umo-WHcZoT*-zKf7*fZpTyrk&_)-e_B|IsMs%iw8B?>F(ECBw%hjV{1ZaB%z4^rH zD&^Awnrd_CIZ-h$Hin0|y7~oQ2j9q)yooC+>35|c#MU8*zb$?oT)5)7uq-Bee6n!H z@9r+@qcL`dvMa*fb zNOoVdlifb&4Mg(8}bh`d3x_P*i>pL`*N-AE2Th=7~e;W%29_KAI z5_i))4H;rod{GqlWhLuODo?lvR^P#FYR|W~`gkv?WB+P*Y459>_$^?5eK%qen0Q_J{Ty*%!pq~DEg2g$(roqcHnoN*@;e7LR7ot^n{(+-LAsC z{l>*|qunm_MkgGW3{FBjSoE7@R8qf0_0<9~N9OFx-R?fZ*=F^VSN%_`jJ^W8Rp|(h zY)~p6o-7iQ>e8%oI{5|^a_6lkPTNgprHn1LcKTpedb|+60UC}+@|eBSvhx}n8>R71 z#C^P9|islN-n4Q?b8Q3bMb-B~Bb*Wk7Mo-1*9KSU-OZ~0#; z2_AbMyAo3KqId3BNxWosK;lVkbhUeZD3xq9?1EeUfszBwfawzC0MDB`R+j z_#DK6hRdptO5*u<^zQg(9-FpqCQYu(_x$Y*4fBJKrvZLctY>x@(-Q1WW$_&@qt7vp zFnPxW2%bVj8ELy;fpADcf)nbCQ9UyTB^ofTx(^`Nouv zOeSNT5KEBf zAJ2-ji*qmPd7zT3t2J%rc*=)tTin^zI8e$Vs>vB;;xX`%BzM9Y9> z3smD?Zp5lh6>>+JIdGS|tT9*5Q9(pPKGy3%NTt(+Wn3L z!n^|Fzg!^lIDT7GCfJsF3-it!CM_;5Zi-hXV6AMvd{}OC&jOD-Hq<`fy!)NqG~TJ` z?A3{2d%KWk%h7q}I^;ZBRiKQBiXc#`2d%u|N2xSZxH8}^h7p~Q`c7h9=I9+QWJ5fb z)e9{BykkRK^LfM^d$XE}mqZCTOy61cx8!qd;U*6<(~9+{^QEn>y=u+qmUbK@MByu^ zyjCW1>iJe|783d_BZ{KfKR9Xi`+Z?{?7L_)95Za_$V|M>kcqH2YVPz+7)a5>EFxbPj;ff1=|kG$LC0OGZS$u%*I*zk7po zWy}QU0}rUCSQ19#tk{%=tfCSPb?g`KPsUXFA6E|1%r}p-QyCqPjY>#owtk=5H%K9h zzW~mzSZeA{nyiUqLA@qE=Wi}+PDS)LJv?LF)zPSt-f7uh6}f}CX~}vKX}xd-Vbl|jec(<%I&9-~Y&4#{UzGy_A~XKf4lxuBY0I#^W- zkn7o%=>Nr#eeI2{P1yhh5sQ}(-Sg^PkU62$*;kzHfwh|~N*SATcJOH~guJRK^jzt~XZ)`uAGsj;c53P*K!_>652^ZCSU=1}C=Ea+n$@2m~N;!^iiOxoH|=HJqP0Vy*TxLewX7i^)YMC@i| z&fQF8r`PNx7Gc+0_HvPuKB>!1*03DL5hg=rK_d zx!Q#+Ku}YhnT=!m@={+_5$s$0`62xu$|cAkz(%dE28a3=QPIiQ171OUoTfj%luZX# z;p*V<`>0ZM1jCl{(_V+m7@npBUd+sPC5`DObsrTY z(%Vp2VzJ|xfZSy!rxlN8K<5-F+j-jEF;sRp*42G=g9YVtNvpqU;bz>ineGU#3>bm~ z?D>-FdSL`ZIZD-t&J^}tFQ&&J&fqC&b4xrY%=<8`%vGU+)fG-EcWMf%&zKE_Ovh|- z%^^_lY5@ldmok1sQIFT;MyH_?zLeP5fP^21wk~~@qx`b95>toX-PN(TpNjHwOD^t$ z^*1(mhb)JiPZne81dx-uBesx-s`_$qna#@ySLb?W*mGC9Nzap} z=2EL(_WlUd`r!&4@W>$Iyw&6>xsMKhftkq{EYCWgr%7P$SIz{;;SA(xgcf zXn^w9%9P$bczi_wX`I{Ra%Q9=1Q6o^W?4Y z?(7QrL!7q*4dTt;h!p?!+pz`d-!Cd~b=!BxT%{JY=nG0J^NP4t%;OkQM7%nW|JYSA z>O?H-hMKuy9ah9NEK;N|S4YmLCQRArmL{lZJKrYMP^%`NCobsftURsTzN|>}5M36k z5VSA6`vR<;^h+Sz26fGwvFV6{Y|$DdPHFeVADbx}0!b+T=r7mbUC+7z)w)mpGz~=v zHnQ{>OahYG?(>VAn#$B-5(5Y^O%0p=doKh6ogNIL%Cp-In)8PXZLVd=6s`>P+4vEp z3ez0>PQog;bI>2(7h?9wY1M3*Z01xD{m})PtqKSLywnT$%ZH*bcR(TjTmwA zm#rKE$*-+|r?|LSw2e~qHotDGbuqY^w9Y0~V-}G5o*iu3v{?7V3!)bSsMwVfF)I25 z#ll5`YFYRR3unKONo5D6!X5|98KHEE2o7IO^#kf@4$|iRmJ5^R`^pltc{*1(i^0}X zz0rY{vw+o#F#CvQx#t@m^2th-cvjX)gK4?b*f(O?(Nte>EOnp0NEuGL%AZTW^BD@5 zmgK*hmiB5tan?!jhhg_$rj&b>b0PLx*xgx`ZfOlMV}fwrqQ7R+!Yx(9IceI5p9Fjd zOX&+zmikdmxM0S&>^YM;-PebkYLlubuhz+df{rW5wp(NxTgIHTsBY)jay)+;AF!99 zxD88OnRKcWK@`hdu8bL_qdg<8?9CeiBDfE@EFmf7R85N+LWhmny^)53{^$cztSMoF z`tS}16d24}bOJ*|g80;^KHrG3c3XGBN&YIs14JU#lt8CB@1o@0Mo@N62UNP%?_R7T z5~QK^t4k6@)+e}b3!N07(wEfGrno=6_VD#RB$TNEGu;pI!Xc1U z2~ks%2x4*JOpU5xa+k*VYfM6!F`yET72B?a7!yE*Q>#rDM-Cn`zUD-Hx)9ZjI0Bkp z51L0Y;u`}ZR6H=72 ziahtHeFDVlywZh7$bis8otMH6h=_AwoHY36a`s^TX35g7SPcG-9>HI-J1Zj2esg89 zHq*A`mFET2s|V3$oP4rnJg=i_d0+1z4s%`W%f>bbvV6yH{CSs8gFopCv_L!d578q# zlHQ&|7tDUY;%GTp$J@f!#U?3loPSvJGU{-Kg2hd~BsZ>HYBVosw?t6NCLAXl`E8c$ zl+-CUakK)>Q^B1gphMI4;%j^B!3#mk4JAe%q&CsX;~;KcMGAoZML}TI3OxoaZvdNC z_{tBBz5YM!y>(oa>(@8BRTNPHC8R?U5fBg<8U_&Ql9n90OS%zJx?5rpfuTW3>5xVk zVF;-iq(eYr=s5SFd;j*{XY)MIdp^JSdCqx1=Rc9*p8LA472mbi_gZU_%$j_iFf)%w zH;OIGEJ`z@L=_KhoVK5S7lTra64 z+agKV3myW++<;a1ouA+hw>SM$&F4=r1Mq>PXSaDdIjYzjfS$lW3;4jfnMmOTJ4*Mb zPR@J8T{=7y??Wd%cQ2J1cMn5Ge$3Aq%J@)_;BJ6aCAFV;@(3oBdOU{IA`nIL>;6 z=cxSslKuL)tJVGv&@up%5o#T`v*k?mye~|P>D^yKEqKqp&iC|!!%)1@+^Q=w259>- zK3|;^&aaIpsQ*#UQ$K$KhFBy~jsFLD}jbc#~yU(0n@8f4Of2JAxx39tHDj+%|Gv|xWLyS%fU~g%J(fmGv z;PrKq&gTqaQ9!6sTix@gj`KuhzFW9OksYp>GDWq?Q5vQeP3C;C@$ct$ma&76cJa)D z&b_l2j=|spvGE#Vi2%)Cey5n!x4SE+2WS@6GmtGiH;9Oc1w585vdiZG)&m5!`u~|8 zAjndnlV$kE8L4T$;)U;?fn+yNBv=m!mhLa4dM4KaeW?n13w}{;OU?W5Vmg70ADzU6 zvL@=fjLNUoS-qR!ayR^?)y(3f8aJ#0K)3~x6=q^^%gdiUh2%e3Dfg1~myy>$g2*Bs z)$T!wg}molys6|r*~~baZf`4HKjZ@8)+&0!BcNw3y3crGTUr|Ujil$Fkp9^`n>R!u zZFlGyp8$E}a>us4<6QxfKK?C1%@2mJy-ttpdSuz^r{-(NRxv$_phDcyj4BL0fYLKG z((u|d>@S%#%B|7(PDgXNu6Y*JM_^lD$B?)v>=h4y+3s=D2VRn9RdPgysHrX-P+fVl=z69gdqZ~ts$p+ zwZmPpftfayA@q^Q{aoAdk?#YZ>K;4YMTZSf5Kvq%{Pq&hUS7^~=G`)XDJwztG%Gi|bV|2cFjC7@c+PxF(`(z` z3#ztQc0)I83?i_oZ`sFg-Kyt86)Yv|dcbAidezRToTVBYk3Km#bwvd#b7{;Qy!(Co zNaq9k)LVYUZkx|6cVWTYJIq%D5!t_lEF0X?%ZuDvr%&Fop7MNjGNU8b7x&N8nIni7 zM>XusrkeNef0NQ@i28V^GNrHQ#S7Sl*(oBAD<8yEEXZHt$!D5f<>w8DiGFWV^|%uC zNyVJ}3XNbI)ZdTcM#&P^wx7x}-%ZznFTpS&iaqMEbSKfIgn#}%*mcZWV|VnuV(ss} zCzP7FIWE6oA|itL&B6YfkpI_=JP@vEAgrGgiz-zDLF=N=By_3s8zTtUA0Mg!Z*C$C z7;(CI(8P$Q3=*(d{3Vf!HMDZFFaGfh5tlu7hQ_7S*%aAE5NHn%@54dhfcvkUWTalX zj&sj8=yif&vg-wTE8S}z!^_o05(q7Mbm0)fOcm~N5v@M!S*5NbWQCqk&$K9l0iSqt z5C-h&xJHR)vL{w5eON+8j2YBd2GpNB0V||=wKBpR5%TJWa!2D{NB{a#5KK3!6~ViM-y~opgpi9FuQ#1 zQi4||RMymeF$?4!WLM3zQmqpcs3?U9#7QI%kg3{hm-7v^v@Z zFar#NanBi)N+b8y3{uas;pJX~*#KW3pu;dP!H;yl5;xp?z}RsoZD)CpVlCKd9cZ;m zdn3)G_}SbC$ANA3#vx?Uf~7dMFpWG7kB$YFkVgLV&(-E0VKYpbi`6p0GktXZfG(l@ zH#)eu3I2J)qSr$|s5iQ@g~pfN=P*#CkrX9IiL=+X_i{u87!)g7X_A<=P-;=5D2K4E zp-1H|#As6UGs@FUYp?ZsuqMuaBOKT^Xvx0v#W9aEFf|Mo_=uG=2-=gy`~3;QPgCOi z%<$4VBe$^$F`6WMw9&k`$}ocxJo|39M-5Y4Q6IZBqija$m#Tr80)&Y26x%MIJIOxp!nO`m#~p^ur8{9gH{LC9FyaYB|B`Z=gs&q1Jm>sJ zGZse5362f_d$r2*QG+xu!B>nLt-`iAXy01=#>e{_D*FA4YI49^F~s73Nz$FpL>haB zLH8!?oMEI(7ljU&SF?mjQTHfuzGD&aw7MYN$!>lDy`W5e45xK5XOb8Dl3}rZLlirw z9~4BDBm@7=!R?y-rz!s&gyW+hufB1my#U+G2w_hvkn{f%qgd9fh;wE?J7L7gWq>-& zjvxZd1!MJVm+Ed@H76IrQ^+v8DFBLx-C6ts@(6hAk1!{1#gof6`;AZVHO$~gGh&I4 z)UD-5vVsT=Z#MMOa|YZr^RY6&_2xZkl2;bzsBF!d$}au%LVAiNcIjg%JfX2B;W&i; z9luuG4_}n!wpia<6v>LhwS=mM%(V*gpcNhmVau9|Z;>Hh>p#8FxFx7)NEP^mN_s!w zst=dZ#azti%;7vd6PZpxJyLV8D8Gu9k<5Auc=t734cH#}LKONd1 zE-w#m4p<`WnH60t^>b{&E)K@nNB@3o6SETrMDBSO(NXI?U;gXf6VUkod>soth#vG# zBVv3!XQxUmlKTh#9O#);lYHMzCwp>v|F3>|+c#(fbH0FXfZh0ourDc+>LG9zFXaq6 z(ynD?aHU}hn`;SSH!D!8BZBuX!G;}wTb{jhh96c76dNANoij-lakh(H(8dNq2s~$N z3B6i$y!s^*_ehKn_cj6G-LA8=fz~f357!WF7Y;dUhr`bBK@L0@ipE+l>fE(G^D2M% z9ZobEU|_3C^wzb??*22z`lyI8u;E?}6Vzd-lW#+8?L35%7j3mj>aI+DEGAa_Ul0)d z_L17*^H9m!nLVTXOf3}bI=@|W@*!QXygjD&u{S^cJH~M3? zD(smOz^L4fz_P^{mA7RVlTGU-;jjhsZ%2E{^SzRNxeHR0d`4FM9o~b59uYBVJjJlB z?~{baKDeS z0Sdo7f7bPu@_*jtN4glnOxxaH>DfYifz$OMsQ;6!nD0T`9%7Yb1_i(=F#t{hieUA> zK@m0&{Wo#|?Dl650L+H|FP+h_6T#723p6E`!1HJF4Ab~YTJQ_Gv4L`!qpK z`e)oEcc$wMo(($ClAJAIkq`h%0E+~|0SP|lw-&>m+twZtxBDVXERbHaVIC2KX`GcX z#|g>a{DlXj?lYGBaG%!C9uZH|1Yj`7<4CA3JX3rjGVzz^+aC$wq!sxmfbPe4=GoY}`ETH?(qn+OaI{R41K&9DinE2xcHnD%RD>^0l?YGm@ zBh797-^ZbRpTtfg6sKs~*u{%N3Oo?Uy5QLsU@xU2l01+ivo4c}#CoE1AsU&rdsn;- zj(oa-cZ)~~i+G9Z3FGeS4O_XjnA-2Av9fC??$o2nZ$?omlcBM$W|@YD`E8*aA>rf2 z_ahoZh}l>J@o60Dbgn(2O}etD=MEJD*95s*%0{EGFmZVyGMc4e~2(ukXSx zUn7b;KHSmd*E$$tdH#X(+&U2x!FxoBEwzuTT>SC|0ga+z9b*g81u1Wo65#S6|JC6qeUOA zR)j8{^JhZD;4R6m2SGWE(H^i?5V}>F3TH!3cjF$cl@pDV)j&60lTu?SBdDSwIMn~@ zF3g4h2VFKq2BMFpucZ=}2oviSO{B|6}y zq9p03{F{d4>ff$@WN+ovZ+XxBA}K&u>Z0Y%jT=F14^eV*pEw>?Qiu!MAYQLhh}}{V zpxTT6QbJuRnr(1ywd@v)ehy?(a4I*6sXb_;ow+E26oQT>AIS8isdn3V_fYcyrqkdh zzC2{}pgJsirpoqMP9lODnC31ocDAdy&jZ>r2oUeD`WuDQLL-y{ z0Q9Oh3Tu#dn(QXzS;L#80F!6dJRxplu)V4M_hCOW^RKNxYZx|eZ z$asG44vP|D1cF&wrTiSsqHb- z8+ON#4({8^jUo?xZpt2qd~<%)3aXpFWo{K@_V(qsT^~boA9kE9@m9{8^*_AW*m2<> zzuFLQlrxhwtkc21QBVl5&_P`WTMmVQKbtYCaBfB9KigZU2}v4TYGME?7QzDM=XVC- zz36f<)SHmCbZf?HD4&^X+%*=LVx()(f>54N$b0G}gyZe$)0qH28xC_RNTkifEFZt8 zN3;nC#Gh8K23xqb(R~@d2^YEiY5lkrbYw^%6&hql>+x-0&yXDY{KGiYw1oz%7fMp@ z6A!nGmW1GH?(1F8APcvS4$GCG9JLQWP{Ks)6$n2?L$5tw7xO#k8Y6qG@7o0_EYgTz z;NlcI!BWktJGB~NJhVIq3x?i)_)N-F0YV#yM-x}Ms^w;FbjRNB-P~qRJ-S=A-(i0? zQde>=k|wfqSWVy(+t&;NK=YC;Ek6qIwl0~oyYxOLR*}T##(F)9z9EW;DZEQLO_t{( z-2$7E@HHZ5T#Fv)(|6#x z&i)oj)F>xOmtEs(VxMmM&e!(G`WJV(AJ~Q6hV)t8Dd_Mr4x{yLU8#Ipe4`EDFna2y zjHB=JwbqA3?C1s37K?NdL6rmEv*jfYeArTJ6lM=6+Lf}%DS5{G0h#W`uY_;%N* zA3qV-E%?(1ft7eaJ=i?8VuJ>jwg70ZwgUv>QkH#m`6U@#AG+22P#_7J@jxJnoon0< zWsSI|;!Ah>vVQfx5y^TsCV370y@1sJ9l6z7a zLyw%lc|Em?#0q?S@#_P@n`MAOYQK}csC{umu0MbxKF3LrZj?3KXraKz*ANg~#{Rye zY=~l;shcll{0OJDI}FjyJ>dRBc)B-xEu@t^KIRAAj2a9z;9k1FrF9lL=lY zR{z+$65x4QREL@v$#eL;`Iw7qoGSl7xrVy-IUX!_l6&9EqH?mHde?Cf<#05~b|wlt zyL+i$;bFmqCT~oRKCI*xQg3Bclg0rue`LR=fq#wQ^|_^X+8-qtsV8KUk0gb>*T3_I z&)Y9m*jJC7$9n?QkBI)%(Y;!QVhOzsPi&K&_XO1Yk>9~3CTT#m>u#4i_p*-GSmun_ zWayBEM8CV+d?#-HWRN&W^MTTy}9|rkHt3Slyy-|KdlLEHS$K3*KVUEGSl0W{$Fp5&YjB!2cO7TYdZIaS)t?g}Z zRw&aX9%K9Tl2>Fs&zz58gIw_b4oohOj|BTbV09bGnYW4N8K0Kx_$Zy`8SmLk2k~f= zhS0Qru4x%(if^_|?Uq5`%ZixS7mu)}Cc%;X> z-B-gKZ;=EHm*T@zbDRyQ$Lw|8w6Jf)ks{hNEKrMmPw}_VZ$EtO*l`10`mcCi%m^}j z1mD+Buu%lfp@`2gK;$bQbX+7g%U`KRe>zX8a102L56mT{J40Fhw`Sz8a`i%m0*$+3 z#PO*37aNlIK@V;MOBkTF3Zdp;msj%XU}k_u#KPSCT0w1MQ|6a~axzm5lt|feLm%-D zU!7heHTg>PMDEjlq6p9{H~tn<_18BM@2^QVh2Mc*APmsG@CW=`%Y5wrrXq@Ke=J@FD%a|3H%SC zl(%H7GmmpyoExmzEk6@STyE8ADwZ_;0Bl8h?x=_tzGv_#Mmmg#KIwe>HTC|*8I+~M zyV!QVYY#hf?NjeC5nsXUuOCsEG#Vh5U@v&a+1s0#DpBBhp8qEM*(Umj=2NP$fSH(N42@IZ!ndMdC#&7VWf64kIvm7ktTx+XBJI>5-a3Sto^@kXb|ZIf#c=*W?d?Fy|7y| zGoF%r={n-|0~^H8gat2la3$AIvmfWfzMt>;Wm(sh+MDqz7E!+=o( z;7%TGDQJ(J8ZpHy1vv5eh0quomu#}TwA?R$1oLsdat6kgPVCFcWC`7HF5UFW#7L3 zqy6hra&!60`GdIarzkJGqsGP4*{qG@uiZT|f_#!io%fME)k*nVbMq;-TqnMBqSEPi z{Jl0_Z&2~Lr1Up7IrDrl?JoYXx!WJS@^;7(-i<=W#otIDD{37K2GcQ*AZOzMnI1dGti zCd6#a=*5PSaaQltTi?6h`$j-_m>dUvQukOlUD!s(=5UaUAUE9-p$G8H7WhH&a^pD= z{L$K7vk`u-%pIFUIfh8*Wr;xcnShm6tVmq&?Vvn-Qf&1c9z$N6WiKV)x1TkDRh2%Ib%tm!IwwdlV^Gcn%yR4h~Kep0h>7!NI4znp_1NfEP&sby3BH z<{cdJ_wV_}zr1|SJ!_skg_N1M=`XM_C)A7K2N)5l~|yNB$CbgV8Qs)LG!2D;Yu8ngmy!@kga*vXdS+-&WQD@9)Wf1s6guchr1Kr>A6oP zcO>q5JJ9t8)-*^3`5|mnGwtPh2=dhGxEqfXjy@hr=BX4E>-ZkyzKgk`RcSe!zu70S zp72eqV|lMguS~~fZR`#{_-Q}s>kr9#Bc*7qLFm(8@>=q7R;ypqOg6+PRdA!4`7!;S=O2Qj78p%=Gcyxn_;z6T*dP$zS8 zba}1{wbR2iBUoR#Sh4vv)A^p`{KL(m5lClHw0{gynQ}Ve$o@Tw*lRpm8)}&x4Pc7| z^`|3d#EmI-HNAVahp!akXgCTXrHU8dE7NS0pZcw!?1DCSe;DY#IEcu!rIt$~wWGh? zpk;VwL>k7T$3_kb;jC$%G5|&sr*C1?0V$%eGWvQsvU|Fh5II@yJps}jnnI``vzSzA z+s{CAp)j)J5FJ-t_Txb&lXM-!qf5IQaTksQYr4^xW${^}1>y?WKL-gfEQ+8DzmIevKsllJAMZL@tWI%j4tP{jQm>63o6dX&>>l_;w3@u26~v|9DyW%c|)4 zTAYEE^%B;@bMZwMH^zOzE(CADundO3bb#Z|ioRm1v&4TsYc@AjZ%cv1nOyqGTJ-Ic6pVx}c! zdI+m-WfQ5NTk%S+8IWvQ4qSF#(dY_$qq2Pf>{I8ZUEzm7x1ShdW|G>S)p4`;s$RLu z@6g3mCuw(JIO3_g>XH65Jo9R4--q&hVtH2Mha^vl)0Ix=V)~=V2+f(I43A?qcH|u~P9@JZH~~ zzQxeM-}~{+QYpx7ntg`Tezc<3yWP?oa88pe3fH&Py~nFo?mAnJB;tc7?y@P8uH3am z)TDu|zdQq6c`hne1giF@A=%lFCpm7cu>_~ z8m={vy;bJ0_(PNolpFau>BKZ9cw^bLMf|$DEKKeV#t|Pd-aNV2uFWT@)>GsaxVbKc zXxBS|W+ISD%;F(B%>Dt}@gG0hu%_a$DJBf?#KniF40lneFw_h)q<$b!?CXE!F`RH5 zzO)>l;+!t_iGc>evs|Dx=&A8=Tz^|BYveTDabsE-%)C&u65HO7t9}B$;z3pJv8K6y zgExfW+iT5Wi?`kqCpw-hJYIvPKdl!F=hl^Tt(WRujP(-S29n4_GxU1lq%>y5j!rVz z4VkG#rdKEI#mL;}<{0%PjW{Kv92DPi`_5AG=2lssvJRW%#>gkIXqF_d@D%3dEMxaZ z)3uXHrz9G8emU{beYE1p6+othbH&nl{8d+}>C2)++#6~9E{UZ)2_#%H9}Y8@DY(t= z*=SWIMhg2tfZh@1>S|p(^`5wKz;$aKq|T$jq=2b1(~~c!eYF%O1G6~PT*+OcYdjT6 z0<&Fd8m1JPZ3Xb=y+nznelEiM?VbXXLx+G%`(6pRHq%Pgs>h4)Hh4MVgec4f!>Ze1HGy^fHyWc_f-WQ?(4o%^Spn*CfL5a+MvH!$>iyO3)c7{C71 z(pT>k=bmfCd~b^4e4pN3pwbj$HpX?j9^S6A^PO+Donvlx7qwg4*C8^~#kVYKZFL90 zLRfCjeYqzW(WgagCF1S!2TaAiTA(OSvtsb&CT+RuwVHyjz}%RTCno0ev!~XaB(|Wp+hMjjlwec$q8{71@cHYXt8zwS| z@rjV+hrXxYqN+C-6jQ%(v4g)j4lcVs@RUsEM$WBL7Crk=y1gnSh4glf{eH0FnFpzg zj^<6%1&ls(3d%&QW+nCmyQD;JDBm7hjOD=#j7Pt?J>l zLeHQ;ov2ckYj?q1+90{G3lm~cRiJ=F(tlEQzaVC^ zyGCO&o7;?$rVp1*Oy|QcE58@H6WLGD8s zG1KA1Q4x|-)#_#k#4zy4w4G}5WC^oEC?|75J^d2G?2)Zh_2xQAkHLru9RGH-!N_Vv zDj8{LSpjAq*$0t7j{H=)T!5UYR>5~KH>L8nQDgs{%$4$7SAn6{_dxVud$o^iM2uJP zb{+k;X1KTifns;8B8ZdVG@&abJ{0YzSskEN#3ne~QbvrNUMk4;$Bu~dT=y1nSslCM zeYoqi%5C3dKVv~%)!EtaA8vzML8}+1w<5%sI^_UKTE0>?QN2m3C_-)`h_iHwJ^f>> zee0eGT&Vz+LCN<+RV%29k&ajZH z?6fw=0?T(oV+E2(G(O~Wm>pF7{&a=ItnY3jt1bxYfquDy!)#e!mE>PFSIxaY^M614SaHOH(cUwfyj!XotoEw(iw3m3%>SjjCNdFis0H1={75TiX4m?L} zlvR&QU8Ek#tsx9b7xEk<)&1a?+n=dUk!h6NLoGmY{Zl|hbnRMQhn41hLq;*YkcUOy zYB-l(mtUB9yx~|Y0B5w}6c;lV~vAVVb33NwR0sG63o`q_L+XHP@+u@8tVHCBFK7qV~`T~ z3CUFH5TSL9H&+B6)+PSo60x(0l*|V^pz}-t^4cL7jZwy?K zAht_Tp9Wlz{Ff7cJ9f3~NmrErrgfxG4882_$d(~OYQ6N${JPWT+mYzC>pHHJT#p6T zSN;`{iI~UVE#QW*qg{y955tbCcVbA_<&&MyE!ke>Ww`mw&w7)mOly0hTSI4Us zx;^IuYWCygrnj!jy#;Dk5*(s`)T|^8{#LRY9#=1!D9k}q3~ztcV5tN7lukT@6$*|N zlpU3uxxQPq6wh7R8VgcwDBZ?23F|-1>1;Um%(kj>364+?2Ny|%?cV-m*cQ@bRU*eJ zk*&aH$C|_fv@Ma@K6%9)WbycxSz?YHlx0L$L+Ahlk56GGy#Cn28QI5|+1EucOVZgQ?x<_Md_QPTc4Q@MKMVSsK@M0g zxXS94@QAy{B}8YK8mgflTe>YIehvVn9JWVSgTs%9Ke}hvZLjk@bC~-~H{KW;@8|Dv zI9R3ydFxx{&3!N2=10O8Gp^gn&Y#i&z~X6syA8R;r$t)wOUQ~UpO<1NpGjv!=N4}Q zR&72Y+In2y1lX?jg40AL#YW}KkSd@2Xdtkb(J%y*^DN3z)t5}IL9MP*uIeB8TG`B; zYOA$4#loD0Y=5i{qxtgmR@vPbCc`-_HnGxq&x9nlfB1?WdoPd?`kegXsW6vWPl9}@ ze!t+%e3S2X??d#8P#%2Jc5<5+%u$74KYD{-l{z6NbY8vGcV2@LkAklLJMrbSl1xg1yn zE<=`s!Xp&S!nRK93i5PLf6*q69XV zt=K%QcPm8Xyt{bI^wRv7b$*Y}sbBA`afvVq9R2jn{;$ep(r6W!r1(iB0>Mb0VW$W}_ zi)O>=7B*;xB?E5xu#T&#n`YuD}ZpFy@Sc4B-C?Td$yN2r2-X- z=HSBkA0rMvNYW%KP}Ne<9h~ZieMb#d=n2*M42NBgVbR9hUtVD|ZtWB;w(07FfsmMAe5w-+b}_QtkUjIARg6VCUd1NTCSA1SgT%c{&U)zqcC$i4fE=jrUCi5 zK05Uc=y^@t1E2aZYEIRLy=fKvA`{tR>G{u^+h}dwj{5q-VUk5>q&9`}7Joqlle@}f z*0jx&lkCuu;2LTMxn$SfGK@%!lnJl@6p10*v*ZnOEb8`%duYzmQw$CU5(dE$5eaz+ zhOPiyt6upog`!luR!&U?SphfMFi68TOIIADO3a*2jZ^QZN5K+@}8NAK{J3hpX z#-?$5HYm@;X{xZ1o3%EBo4v}JHXBIOkO)5ZaQj^rCft2b{}n}JX*}4J+2w&PcV1lL z+*}xwHT0VG_AzHamn~q7gOxxEX#*+@ZtR9$#+QYPs$Bk78Fow(j>>zTGtP~`xoU}| ztd|vo?N9-Q@BnR;Wj1{>|nQT%xN(?ZNc|^^JBTC zxAXRC%j;07wm>F^{4(P%idmqqE4M^cqxHlhhhoFtrpjoeZ#oh_&SyEdWt0twY(jv> zGd?qFOkUV^`3{33RU*LK=|cHJzBsUG3(|)}tyzI!B>~n;@eu@II2mvClC9zp#E&py zh$E`2L@}xgHM*!=J3^Y_@uNw}0hk-_Ui#do9%@w_%Ft#8;MqKhHyhgv0;)jjF$>7R zNW4Syk034<>WLK%CQh3hO#BAfsM{GcE)SZDj0d_ z>P=JWaeNO7c4x8skw98HFt^Yx;;B27u}p9uG5aTcXs>cB_c?SjR!)m=|VsZRLh--R~p-Nix+q&C4>Z=^%R&1LxHVS?N(t<@g*!#sRB)(5wN(oe$y(NHP zr~u`9ETj)qwFHBQAi6u}R<)#T&WgMKxk*Wu8iX@IK@Vg=e&J$UXnd43p_!DbYFsMs z^Ur(hui_p&uUBYI`&Fc$dC*?>NR;7X-aYMjw=^CIIPF=91hXyjU4stCa6#g}lqc7z zPDx+;gSpzFLaYL?F7K)N1@xymAiapr)nn>%K%GFsZ{^5JwTHbi?${kSY1)EUX94+m z^NGQydTi_CjT+NvM>d@l=P-sk4uz7%pT7OWlT0c?#A)42cC(0=d4^j+=gH1^W{oGu zQAG*0&UQI7DXR5z-IMo#a9f^Xs9@G(Yi@ddOL$=BOU^Ezkwx%OLCa(R@Y2 z)x!OLR>cH3k%?8Wz(X9&AJVy7ePD*hJ(s!viJOj)HUMC{moFw3Jbw9j2Pr z2l9G2+jnORPT7{90C{Gi7J@H8lr14iaSA_}cknLrKZrQof_sLoS89YMC2&B4;xthY zzs~2iJFFe?=~k5hkrM6$kWc+6sxdPE(OKH&pp7Oj4ahdjq(_%*4R-J9`r>~*l@x z>INBy@fYcuL*GNS`|`=OKrCuU^c;Ql`;yASN|o#4S1mm11B9CDqh&|pzOdU5RB8&H zR(4W}YKvbfOw>3nz570at`8#@`pQL~KK8u-rCN-TCtkYc8ePmtzM2Mq<@$ZIzSIO$ zmJ^B$t@<}f2lt$e{mI#nzJ;)RwEC(|^=Af}ES0`4-CxD+O7Jy$Q@S~@*A+}K6U#4o ze0-D?k39e`3oGZYaOlgG9Kk08$=Gs8FmOXGv-UL0a=H?Bk=z>6%Ip=&>5TwzZu|VK z=8Ag$c97HBS20wb@j#xEG6{gI08qlAfit2*A|3)y!trL&;)+wrQE)HsKsj1&s(zEXkA1H2wnZxf`!FyD59n#-Xst|22Vi_>;Zoz0o> z{W63U;F`{KNtx-yABHfF=#nUXF)cjuRt^8Li^oDTJTl0p*J2Se` zS5mF8n9aRD`ltfFfRZk*j^4`KT$tzvO4zB~61+R;QU_dg4j6)dk0iT5?4NZ)tzGRy zplD`DV9uxj#pv$kv7PdrlREb;GQ}@lGagKDs%Bg{psAV2S82K$AMJ0^@qz1)P7Zz2 zFEfw;*)H&sva!D~CTR+jGJmOM;joYw4-~SwesYYZgG+J$#)SXlM_R7yUzap^grxqB ztd#$^KI@%fzG`IRo?CGExa}V`hJf=TvzwwAPR5c(0pTNK@*7h59#|WX-1?;2~&17z4Xp7tktdJW>3%t4Ch=fT%6~ z(bfL?ue+MGMz=HZotlQgUuGj_GUw{&5?5DZfetph@!m5+@^b^i3&slzTe>SnpH*%6 zbcA!Pb*nRK_V`5It@9+5w&hIU11Z(`U|L>r>>-NQ)~VHf{T5ow zO+17%#8VlN#svAYWY1?nny|U~;y>$p7Kh)B5#xACEjcuA{$u*}p&kaj<9oj8!r#Vr zdIi|JqEW}|iRzj4BWvZ48-3TND0JtZy}84V47q)HK5_7-hu;f=86!Y*hGxHhoD$^b zF1p{&jDQd0B}_utRLTzKfSVCJIwQ;8aOjE3bcxQ%Zkh%|fB#ar3LlyIromeg@Emx1EnRsXd`iw8#*ai zn@u@UJ7h+zaVStmah!%txSDw&u?2Ch6&!QPK<>={&_=5Y)pLGh&I1EUMncOE=O@TS z+;FhsNJ}KcNQ_$dcuytgKRE1Ju(N}~Cm~P7#VNX$5{=tlN$LpC0*P#m6`EWgnUHwT zCyk2x`MW3Yo&BsQdE47;PRz>=Q{rQCzJ$k=EcSYgqkPj6+gvGSbB-TP&dIwrX_Og!%L*NAGx2D0I~Bk984Qs8Ghg_OwMa4TWc^QuM+STV9; zjT_K*hn`PV^xDODYidG$(_>M-2T#FzNXfz9?kua#i*C`+ks42$+&acEUO#3zi|OhnXo-#9Bsv6idf$tb4&dZ5oxyQ&dIJv z-z%9T%eIVtmlH>?H>dG_8FIaQF0?pn*T2z+KE3_G>oFLThS&}*lAF;3Aqr_YK)QS3dnzS)jx#yC#)gf@^c9+c0d}2Qqm}xzL(8t+l~^FwKL9ll|l?4XWoL4_C*| zhR-RG8hu<))zPz?{&{faIvKDF7J_=u>Oa#2FYkVR$hQ?A$2N=~OH;`@JU%GAABtqE z;Ah>Hdgp3meB~V5dR3%J!?12NFb^7)cOWuEOEX^}k;13^aq!YKv}ui*9qAQ$2&x9F z?>bDPNW$EKY=uQ0M2K`gF8CaoqrF*u=KC|l2k6=_mQ(#RV^>LMRE$x@g-7b{@l9vN_Y%J zQ!dY8{F#a7;VNU}+Se>pO07QG%trGHy6z1r%JS6 zFG^bXYV(B|s#L$znkgRLVZN`rk-J=(RO}!bZwq+UJDS`l2K6j?K$-LKnGln!+s|+a z58H40+#|m$aYhifS7KMo?~tvN35&#BZk3m%+5=)z+%8J@5b0y6-Ac!W^~nY$8RS9I z4j17}*#aXgy4KE#zA#)XFS5beEt(sYHROqRWZS)M^ z8h3=oG@kM=Ia4s7SN$x&Nc4K~sG_aU4~vf_verVZDOT?Y++*AoOMmvuQk00UC_H$$@;?}r-q zs6DYmZ7f;udu;Iv!T)26t2WAmL_Ayb^?Fr2KhV|)fX#(_vO8WL`<-^A< zU~q@qw)PLC_+*~g<@a6yGrkep3^XN+v7H}gSt_!WGjTlX-;kEvw)My@n$WFJ1W?Y3 zO(NSdOKqd(ojE4ZZZhfVN?+BSf7>LwVVUOtV(-19n%uf|@vU3Mj*TWIDoyD{MClfK z6O}F?U5X&R6WMO0gMc(CQ4s+tk=}^~g4EE9lqjKxVkjXbA-OBKzjMy_yWu`yyjQ3sdT64|y%sHQDF4qX$wY8K4;<)rePzMN`o?kOH&+m19KP7y}{7qlZ z(C7dtjY6;LTcKfZQ??j(8SZ0^bNNla`YoM|^48=b0}0#C?1FpS*-*-ADA`qyrN({h zjjCj^;9X+@>CA^bE%yD`lo+@T&sWIN>bQaIQ(-J1BFW7O_6n=HwEL~qe}QLbK+EDm zszx*+&4X-;YCh#(;v@uXs3@zuVDvuXklKg&;}&tO`&&_`gjU{h$p_+KW$w#Uuw-;Q z`)m<6AUUNdzvo`|iPA9Q_leeLGeG1**57I0)=!f={w{V9qAarZ53^~py1!{-cjLcl1}he)3nm zeoNi?J}q1a4kyEa%*98P1*9Nr46ZWe0K~$8_*t<(NQY6*!1{OQ{-T=*3o2HDf9HqW{GAEa_wHI1guJ|q5ivWhWByOw zpgvwrzT%+2jdhdn>E`9~z^(Kg=aJRb76_s}#UeP_Mx04N?i}FJ3I+s!69#lFZFlqo z=ZhK)deT(dMd${R#n zUqx(IeFfAO8r_T6b-A9_Jcs{%M7V^6T?db~Me0r`jS`Fs&*;pzxPe3&uZ7mPt*2>a z+{AGh=YQO*|8@+A?+f{-ZahmG;}*;d-aBm}6#DhvDC~ObJpR?+XB!PZ{uJhE;?eYE znV+gLto&8u-YK9LjXIEWsfw7_Ka>g3>dVpO7ff+|?m6G9Ifi$&?f$O7HM(tlyLDo^ zeS!zvIG>E+2gOh3xF7-f`^x^C{^pHD;t@7+_vbB@4}|h=EituEBxI{Hw%#qzw)Bf6~XJ;1!NuW)A2H11qX5_9t+(IQXt|R&kiBAAt`IK zDE^TSKaXUMEIB`0pqaFp`?*Jbxm|_s2VS`Tp!?%V_Z9g~O;BkssG(8r@tme?LDeZ{ z*gxa}P*)V*C}0?ct?ufwE@GQQx&ygE& zWhs~b`zDv68m?vOtcsN8h&y=Xs>I{luBO~wxyU9$^ASV!O6P0;Xqh8IDoQd z*WweNGB6Uz9JNoP(`1*cH67)N)g4)~H;;1=#)D3;(e;hDGL!~X-)qJ#2W@=k)l3n7 z4ir4<+y1(0T^Tjg?M1NGyq=pkytRG-Ze!oPU9W9A*RgXpg*}!MpD?1~tR_moyf|g5 z71LYl#SITli}>c44Z`k>YaSY}{|KG_2xFM)NbBR8&g&U!iSbj`ND_DyjaTRfouMRy zzi+A1#IhwYlZ$$9-jcxuY5sL2-cP7|H}-k+#0R3FVVL>{sbL zlJofWh1I?+2yZGkQdKoK@hF=_xJI-t&`og_SyXJUAVlZv(kB{KqD75fJU3qv zV}etW;goq#KgTJmpdp3*y2e1xg4RQqVri_Gs8;jpWKI;=e|nn({uG3NG4kg#_BK|< zezO-A&~a>uYl&6!^l(phZ4noHpwLAR{n%?n{W}|}qxBVLoCQRVe0;SmAuoA=?##c< zd$NEsDRF+5}s zN&NgYR_N9$?>0sQO_8@95+meMvF)iAc6O!P|iDKduoXkd!ZB#Dr-FF?u%$y2= zxBWNf`S^9S#8hsfTzHwWs2TaLyWs=ZZQv^oBZY=y(3*m7ytHV%b zA_(JE5t~3bAu%0em=jQC?!G=}>9>$($7Rb<+x1YB_sC|EbleZ7`Qd0Yhvk}%G{c+Ei=|&RQiwdkkn|0ja zfZ}kz-l(jnqczWW)%GXBH*;?6-}pNFaGQ&%eZ<4=D+yAM75FmRL98&1Ms+}?AM)tO z)jawvXdUBTH8%+)ZuzbsFPf~&4B2CJt@jbZG_-t>2H2I-dNo=@&|{2l?CNm(70 zSq%g2#J*+Ao5^fGe<`)_wRWSx2HA0mYxIu_TGN0g!UN@}?z_5`&t^MrMeDD`(^+wa zE?VERdz^$M6ZCoMmKQHBa>oxZ3r?`Q1+|a@roZ^1zX^QSibfL$W@-M6o%?tY|Mmen ztlnTM3*T27YidSeIRXdxX3vk#DQd026MjHRL5LTE0f{!^DiU|vQRPmSJlYoADEU!c zF8>b+6S^@!i}9w7Rwd-sb}spZtwJ_e4WM#~J~@tJ%dS{SKGm!_cWaqI5JX$c_J z&20h+qlll5XMRGUW4tx)v+;m2wcxJrPH^T9F>UBSsA-o*qm9W21SecjE@s;JboUVD zQI^a5MEpRbtp@s{XbkLaVs#wHH6ADz`7TY_Rb0cDty6ql#fm)cqyJnrT9rGU4;Dy# zorifzChn7e-%!;=plm<;`jk=5IXW-cIGAEvWn-8<%HG1o#Ke2roGmx83*La2@xyFf zdp2~)p=Uwe%a==-8Z>E=vgRl@ zm|k|QxOG^&Jq%j7DVf+i@W59z&ZF_RF4n5gSa9_k}|(kK&S?qI(ux>y~sl z%e*;FXv%sowh4+}1WsTC0o1Ji9yAYPp1z*oJW^`BqUh6gsx_S-mdAe1I}kBm?KdfN z$@tBS7IFWA6#6{)w{Mk}S{Bs9j+@!;D6YQUihC#b7^sEK#)j`%L5-pry{dL;Ep=~J@a6=nG6B=sL^#RK(DrdQ4EyIQ zU+2QYl=DCykl?lhjYG`a89@oh_1kPRHs!eAJF1Y3VnLqi?3W+%=w5NQiZ*Jzkt}TS zb}0Yty1w9*xzYuxy!_K|2A<(WbzT6CU<%-;x8ql=K#B#{mFeG>E#=hy`X)opBNB7* zMz(4LPt}BbT|H@;FYW4-UfRHz$a<&Mc2}X=VT|v9Oe3^3x@%6nNFKwHVg$gUFNw7TP%$=cOX!ZAnZIKY*9PGS|Xn< zXAZBe(a;3~L$p7ZQ^J4Bm)p{ecSCxx730`K5|GQoyi3?ls_@VE&1XOTvf|-68exgJ zk@X@Mw(#oo%7C{d-@6_l?|z}Ka-*?!>-zY<*DA5l+DDSt0`Sq_%&Y_qrCglnfSfBc z{$1w+sD0e2)lf2&R?O3z?i`v7b&9qjJJaMIrGTT_0I0xNwQ(Bru-Le|BIHW~?6)>G$c6{V{r&uxgiQYWKti>7qTg>mgj@ zB((um$GH%GgCg^po&=zkZ|z_^a%uzSzc}^tkND=K&qJ|Gc}|D7S&hbcdyFN&SJr9f|NTs4}yRJGk( zgY7Hc2G@dJ&}1%>Qn`b;cr43T3l5X;tE4Hx9lq5!3Vt$ubUT95o?JM@9a-hrr?yh( z(kY9an}N0`#$e}c*B~A*-c5nbIZ%c<48Mxlx-zsb%e@5!46|ZeS4j#V(&r&s-;o_> zTA;=z3)`Hioy^Ujh;t4XXYMOqi57uOpT&)3|5g>M!Dtvi(U6q_Z=~E% ziRPH9RE|EmAdQ_qpnIQFDhTNF-kic>FmT_+T;?}#-o)?_+wUMOk6kh-tX7yoL(!s{ zwd53MI9{@fH-!~luCD1YSYF{D)#9`QSJd3(;*=ZYz5i2MoKj) zeb4k&XzJ$cHB`pGQm|ka&Jv}`#g0O*7z7jN8T^WU48PbV#uKP1N7*U$@jQygJMG^% zr9Z1(KtC)*e6KzKec+I*TK=6JuaQIJFVg(eS~m|R_HH1dRJ}OwWwh%DZ-WAzEIAQK z5ILkHOHy0JLPth#98}xV$jjz&Raa88zZY!xivuyE8n}0fXT@b?iefNB_qabO7=mAt z(aijItZ6$x+3xqXSm z1di<%Z*D+9rmT<70Xb^M8Bv2fe7Y~s|0U^t(NYYyS%Kc%fcEc)EZQ)OBa(*<1B6H* z48loS-y@vLd26^LQcB8&$hXb)mxRgTA95N4(9r?)NS>(lpB_~qPv%%qxg>r}4=Kk? zDEN1D7pSjN?>`58PxUyL26-2W)W4(-9J0Y3;nV7IG995HI$qG*Zs5&HC*?WBa$L*t z$mMd!3Rhv{X9j8kM_+x^yK{;VfpP4?3U-4e{Tyfl+LMh=0VHdiwim`j9zVNby~RPA zaIaFj)rgQ6phL@j{j7jWgm_*vi-h}&*;nU4{Qp3X*Zt4n3sZS4&z`-_2V3uE51cFE zY3?by;pdL?7TUr31!919a0-)s>#y!hHwQxC2^iPdOPBNn1jlAHW6mHj+YOZolln=Z z-U{@K!T?vkum~Q>K3y(7g+$FxPJqHSUjH8PWrqE2YC{E=~cD zTb{r6R_(_d88>5Iv2^&{oM(cxtN9idj_~}WCg~)mQ^K^$;f2kG%J)v=FMXz9P)a;F zbgmQ(_Q_OWHA&uPJT)gV@)I_0pU_PN(7Qve^Q@Q6U-506319tXV~rFM!62en3J@#g zSH(cx5?(ZvWGs$tWxf!FMn!YT8&$e&k5ym$u|Sj$@W$?Q_h|4>^h^9Q<|k=0{XU1w zi|4+Yi1v0Lzy1(y^BQ7GtU~)~@*ybiNp^95{HHMEM}!QQ?SRNPni74Squ;a= z%7Sf7^P@YvpKrW+)srIAH+NSaO3AqFVltGi%^en(IO>XM-)K^ci@E*}v~-SBqWNE- zCDR?Wlorowp?%EGg~v*DIUG0h>FxSuORIWE)O)6M9uUgZIsn0|#3`gxD--t~^eFg6 zV^Q?J^+bN%xy=`sg0g9nLr`||<3S?Ch_*G;Go$w-@c#A~*lgAkzj#u?eEe5hm{Oc< zj@n3ROM2hN0C9;=;g4nlq=cCO=vad$kao-n+>NoJlAl=8+BdZoz+;B6vt7mlela~u zhDGLl9y2;a*#3uWM`qy9LAvLevK1NhOYWjcUDI;b^R&xPoEvm;`RX!M*wFO<3@|e9Bm!bZYjuf+o+TSHmLG9RPnVeSZXWO2f!qQ^S`EWAd{$)GR5#8u@2I+eqWEjuod%H6jpY5b?zl2BT zTYC!t@Y@&Q2;BRL{G4^3K;b~B(RVNX%Do5K*5vOF6TB`9q1aJktOVOB~z{V z6nvj_A_AmO-ox18`XSKTQqFNIr0#k zN8DLea-y%OwD->^+I)Fd;V8X*=N(AQbuXc-T;ZNe6FP8I0&kR%S-lRbpgNCGjp*pf zrBFzJ=(QZrd}8rqf~1^O&Q z`}-Jt>Bi3|Vrj~k7wV^Tu?h_s74*xq;csnPv0jjuXS;76TRizbWe2$4Wa3=dwUl>I zdZ&~pKKE~>yvkg=3+vsM+cbQQbzhz7wC8y*+Oq~i&xNz0bAv#3*u&kzF3l6j{WJ{U zH#+ce60VhcA@ne_cg2rkT=tt&1yN@j2LV+M``3L*8hSVEIMzboSSeq5%#3O?; zYxVU#?u#aIK-W5}*drDUUFB{^OgIgSq|lW4cyar8+L^hDpHqa)&K+UtlroGm#z&{Y z+W`bM!*-_q_=05RCKbp?q!!RR%1woqeDS(J(DqjlEd8kaag3qCc z>&6B;Pk78~ujRS^NEh6QIr7(813#jc$_l$DKT)X<6ttd8V|U%B+c1vZBOdlPgz%8n zFO0Hg*i^d9bdF=cBRIi;5G65i>+sx6ey(gVxs^epnbn5ph3K#SD{-SgbXOLV&d&bD zV*&5XCXfRZbkrkt2h?y2C5^!RZ-yP`9pRFCQ>8&v0Lq2?HSQzU7E%d>PH8kaNr+K- zJ(x`Ye7zsKtlUERCx0aRGYJ{p1`=vv&`Irl{`j3h(|LB67m2w#UqKZAbTApnzBOmG z(x&;XZMFmri^;GCS;my?_iG0z*ykVijYunpU3~TQ`yU4}5;BSGYE8%XaqPQwZ{IDv zveKD&{o7YY_Frc{dBWyolUHD-9)peEO3ab)@IU4E*u?MKe*>YP)W5-JQ&Ux2S6jQB z3uAa+SG&-E4d%MJjp-N7RSnD(YCkq{f>Dm}@w;MiDk2_S1g${9SLM#2$5*ko$;Nyn zISMYWil6$?%AkG8;mQAe@tsoI-ue0wf-SOTosBWYZ{(8`Q`$#ER>iNghv;7b5-kyO zVQt}BJQK#_kSYD^WX^Tlgf3?|c`V=fQL_)NI}d6#Ph50)Fh`ns?2GvP|ut84VfJI6-*q^<8~ zkh7;M>EBqR^r~~OGT%=oWYx8Oo%FADQ$ig)lV*rw7@AAJoWs^>x6Y{5xmn@mv=>p7 zah6+&-Uozmow_Iy;%eRyws%mU{@5NCD8>72)cwD;2e5ru;>uE z(%muoe>Pa!-}tN1meg;9>#&g_{W*D6v{!j%VD0#2V!~&_x-tS->*-ysmH97gy#YT- ze-Y-c?-+;}dr;!L%h@A^_bdZt&N0&e?YY~$A#9TG;;LnL>?JOBm|R5zZcdozxb;LfLFrHDj6={}b2-|wykJsN-c(N!=Pxy$AY z+Xw6=-Eigg6R#xtGyF|}wy{=}?yxHIUW#@Vh(;;zZhf?;0qZ;6Xt&GzW`Aw=a;v|` z`XYi7(_>V)i6n;AzwJw({5}3Gz04|*M-a(qB(9)ZLjQ`wJ$siC^ z7zaUUb!1$->lRq-vdc77Fst#e(C*JpmBOa^N*K%8@i49}aQC)g$eoA^Ea%4wCvU23yDX-+si2hwA5y#T9^KxwXE@Sl<6Uk-itF zKl#7hsgm?Q;M!YT{c$J3;kfb);j{~k*nMHMx&*fChsns2&xhOjx1Gy-4QF?;nGmj%50Qzw3QW2RxLdC& zFAYVTW_akoa?UtpaAu&-(x+ZbA+sK$7@U@R&BIQPv{)vuS%jf~^ea86-m586V{Wi#N{uzpyIej+4 z-738h`-V_ix*K{D5P8nr0#6m<@>nSkQ0>19+|Z=E7=0@%KX*Iy4WXXo;6}cqDST}b zypsISKyAt7G+$FRTi@unce}uiCrY9R3mb`NTlZ1QPkIgfE59myu&{nrKsaNZau@V0 zmIbe?XNhTiE2a?CgDzphFd=7$cLC^PwZJOlitA(O1E~hS+wji>fLeQ;1)^6fbYD~G zue{Rq{xe8Jg$tZK^o;Ri9`kKh#Xkt!Q`mzaICj}{6<}jo6kI*_jPACpIGtx`8Hk>; zdwHAsu|IZSj#wji;aU1fA%yD$*rUInpT9tNaa2l_IuB!V0YK0W#bpzIe>6wm+Pc~H zgyJ&Y26n^Ws;}%qaR$+@sFoXo^q-*gz#Kec&EHp-ofwLC%3nM=hQQI z`uu0AxV69Px50~a2Y(QpxKBP41ah-o>A&-K!Dj+(e%lf#z^gsHZF>NS80KfLAfkUebw5?P0B&yJ#)dv1oVvU;;eb#i} z6wMbxWJ2=KZ!l^tsXcSjPCX%Mr?sLsx{EFC*{ct1*J9dpJ%;|bf!*_6@|9TdJ|i<3 z&x!$qI90B+o}7X@`uJo)5R9#HC|}P$_PQ2t+}XboP$xp7pf^_ z-4YuUkgF}&uky>E_c37@4oiKgPK-Q*gl~wRwi+_qynny%=?ZAgn&|d9qUCra-o|&Y zH8MIvQ?y1V*$mH%8%5(=>={YxKXn#=>VoMv3?1=g;WHZ+gmLb(?d1K!OjLtNLNbEBa zjz+sjJ+kVQvdHq<_{0DSU%Bt2uQ~2gB-ZHaZeTeE&Zit5p~a~Sp*YtdETU!hxV*ci zcH=`TXnn?F`e<#LWvPnWs%APlPItM|P{nO~W2gzTQXlkudb$#D3PrS+Z{Lg)LDopIhzHE)zeA$E@34Y~ET6Wtr?K(~Ut;x6`&nr={pF zhRLamWnz?9x=;}cCNn5%X~k>Eg405FrSL!hSYQ)MHENdRT5LVvaVoiI?LM#B^}w-e z-m;+i!?-egMMVPTuC6>&z)~6ZyOS>E((~d*=nUJey=*Tu#BE~Bp1RgKH4;qX;$VMN z?sng;IR`gxlk`eJZZ(~$%pMiXpXDou?FG0y?aWqg!PnXSYn@lG2**;VsZ7hOEef@o z&Se|kBU_|pM8H_krdjByJAF^c%+p=;nXon66NX=U6NQR{kr9w%w-lOJ^D#B@}IYz z%zC?B0H+v`S7+E6Mmu5kmuqJDeZ~d(w0!l25ad-{9F)wBMHySjOD!X)jpPrXmQ-B7 zT~(;<6Dj%TSfE-fL6k(vokq+fgd2lD)jSn^$HpmZs@6E3%Y8%| z-by5od7zpHA%UB6+DBX57f)GZoMz2A3W{rq-)pFZ7Bpr)7ZwZICIq<6+p;h&L&}3& znv8rh4ORU+!?^s#x~^@m6Z*`2j&tpjC}rlkz44h4=(71^o{`t1ceY6cMz~kF{oZS` zM->t=zDe+1-_VYY3!XYX(tZ5u<)%aRkR7}VXZ2|?yV`qmDjZ(cgeu`Ton}$$zo>`j*7&Wpw z5PP?b?K2?MRp8VI3ysBPFQrL~zA(ftEQF?HFufAwBYSy%GYtHkyJnXbiRE5v58+O<$McyiE z(>3$h((5p;@DyI<@h!}@Y1dZLP!(c3({1j27=K-2`mr_n-Zum@?{RUvu9w#cU6aLW zU7f+xQ)S%3l@{HYF!hpcHv7!dS5-43hDkLu`i92BUGlSUu8+RX1UE~1b9gNNSQGM> zTn|FgUs8vVmrO`qox!L$*1S)VIKTMK3`Ql%<+_!LYHA2+vI@Z&jzt7$bNGIIEsj%^ z?wrsuOx{AULz9J)(OURNSB#W`k!k!bjL`Ll#neFeV4Z-IP|6z2iWnW1s6QpGZt%re zW>nr|rmtlDIOHp1M3gAmps>7GU<6zKm1u~lQN*&R2~MRi>5P=k0t+C*yg^q_@G?Tu|B zeEi&{DXzpN6!gNT{7Nvi9$x`x=UB6#5{*hP5I4(`{DUb`)YjO}&S%U>QQ6>;tJHc| zWNCXNkL(Qa^(cc#3q=Q~rHUbSywnBZRkYKJ875?^VO_6DK-O&!)wI7eavj;hlwy&x zrMT8nIP6^~T*|Csl~!4@?Je7Em#)KhUy8yK(X!1r2B{iz$9&iLX`xkUp*ZAETo1#{ zOf`7w!>SrL(AJB?ZmnMFB`XwIWZg=m)1K@PQFP$Np1mgCsC~#vtJk$ z&&*~*@p7TKdGY!BMF`b5K#nw4=C&>l)17%G9WKi0(>As>Z?`(Z*}4UdjFAr3ENuLC z*0{_ey>7Yk?i6BJh8nm^9@1iYb#3Z9%6|5p#}=AI*=YSkQn#6*YDB@{zFUqBR6;sk zXx)9&f|9{m_JOKzS$u7K|Ft);l3`WXvMjO*}4ED=qmXgMRk z7(7)t!XCFx7r@W`Njtclink(%dD2enK&mFZJ>Gh?6v-iDIoUUL(hc$=RM2m2nh8^S zxlEWoT;SH9))z!6Iu^NcoR#QHy-7~U7&D39NnAOrd{7~Rf=!4Z!EI2k_4BiCV>nei zRG3Z|bn}@LCc^{u6`xCGuRTiw?%Zc^aM-4T=#-$3?Q7ZIM6<)bdCV^Q5fO3A>k-2j zFXbUHv5HM27V^TQFv`9~W>aTQ(SZ4vZo@wqTEKChxukof3Qoaqmg&!i6)$HMH+d`@ zinhh7yB-K!w}za}xCrsjelW*6s9{jAC~3T<5EM9DTTsf*HWF1sHN9;~j)hQ+0w0cY zmT|jiA6|)*nfdVPu0koM#d?%Ve2gb1o4U5S1ilqjag-YVzBg-1TCkUiygA`hwHQ8~ zVxe#)5R-15+a!)ilURqaMfXcQ0xuostsL>ME8Gs+)OC4t^wMKpYRRXet}PY(L^t-W z?4a4Kl-sJhy^Xa0GP#e_?R)NKTkM)5-I$?6l!lYmQxw-MMmfIarb*f(!xT$9zZbvu zs8igAL0VdK108-gthaW971@8wo7&%UU^b_3voC3U$ar>OzVGUGbYf|24O=L**u{h-v20rk74#TNOPFkic^VhAufPMc~ZR&brHLkB>))Z`>hHm^K9903NXQi|bFJvCE9zOQHJW<>TdZie^he-x@Y~3cAMP8}#3bQ8Zhiya0ZAwK@_-n+8Yg?X%K39J03* z5Bn~bsA8;TQI~{7t)$9VZo+1z1)nXaQ~ii#jR0ahRXX;NSEk`CwrN~td7lYIRYZ7` zf7k4HjI1jpP64kr;an+C>VUThnwJ<%De}vq(#`3n@E#N^RUO8#Fn?SeLA5YcP6Ym3 z0tNsX!#B{Nob`?Yd?vhZUfQ~D7>p&V&05*ZNysadv}Y*muWlw=1cwr4q>J0kDT}j; zF{gFpt^_nzJ2g^+Oq*TmmeV*%-8UJV3{ltQN&0>D<3C&q#D)R`OJ7nBU=a##+W~=( zM>iYtutVnCU!3DJwxIHOWrlC`fbwr4#}D-vizyUEL-O=2$>Y!(K@!`*{%J%{`jsvN zdt;{}L%#is?R;UEXRbM42Y4HOQB9Y%HlaOUHAuvTG|*`*3mfx-`=Ut-V!G*NcAXl* zXE%o2W{kht>IeU$7eHkneP097T}sf}%SC&$KK=QyVvZLU=PaiAg)`8Tzuzq#--?NjH=17R zM3{|%6Xd{W7ueV(3Xxzjd(Dd{c|IaaAX;(ZD+Z&@?jom4I^r+h|KR${D7_N*$2sOX z14}Ykm2EX>yeCDq)`~nZbYHw$7B;F4wUpZS>feIgJc8LA!=83q38Ip>zIpE7^gN%m zE>4BV%`Ck|#LBpgRr~r9Zep;R&6#wzdQZm~O(m{Yy6?4(EML3bVp$UwtO*A3tGaWEuDkuLIe- zRX;lp4&+#fe^IzyASU6+WTc>Ir*|&ok9B@a_89S}VL73L^Dz;&$!svDRz~4y>a1?> z6@Z^~xa92hbV2&SVtZ+YS6U#sBRJN3*38>iZ_Ppu|JHXKJCVAC7BDuR+iH5OCoREj zyYU8@$hp-SS^gVkT~ep(xZmAX#EK)2+sda+B(0-NM{4*tR z?K9y=Tq6q&?2n9Wp^_@I#S|Dl2lNf1pwvwtMO**hreH{KHBf@mk6 zx~Uud{7i>Ma?#<$<*WbjEMQkq`P`!`K(T2|`N{E5)te$Fm#y`dO_bR7c!2j#kw ze-RBm7?^o&s2P5ycH=#WWqIsu%11COHM7gZ0qF4qVbc9rr`b%VAv5U4UNn2Gt^eek z!4W5jVEe#BHQby?J^HL4GM8-=iF3G14$ z=m(JJ*xP~@R{}C-GIJ$^mAVBq9Ta&Tskk&2wM1{uIVlO|`e0Z0anPo~70dlB0py!0Wo4 zCSN3?+Ky`<$<>u#Y9g2WP74mj_UXz?2aIc2lFvHTUrtL!Umi5JA9ub3Xmu;>^~ggO z3c`b=p^dU3299b~@_eyuBKrXkgKC(<_K#uPDo%ztZPzI{e#UHa^shgH_0wmIs z$8G1@`MT1ZI+V7Q$y~*4w_x9T|M8)wR{qSidY7h5p9wAou5usz#2Xiz@&d8oryCZK z7Ko3Acf&e_)wymQ+$MiS)CRuhc63NW-r=Doc{YJv;%*|lrI?wUS7(%&GaVA0L9Ed?%!skO#yzTFz`^XsPp!2rKj zY0+?3Gh8Pyv6;la-zT^C|$~bpQ_S zwltQFQ;^|4PU#39(SEWqXsX!g+o9g7aRqas;o8rTB5Bm&TG7UzrX{@^1w1X%*63|- zl))2SC>K68yD{U;5JJYu9?b19FA~Xwu622=oUuavPOK)cx;CYXYc`3iLa&)qR-vt3 zTWt>fO}fFWUl#pL#Gunv#i*=YjKad)m&tsaBr?cVoEuGHc>z2objb-s3=nG4)V9jo z0xWc?zKnajWgoS&bul zeMy+U7B%5M4?fpQzs1BcPc;aFG_I6E-SDtNa`%;?S~#%SvX0(Bky-+ouzU<1QGO1Z z8@I>-8M7lT0_|3Ma$vGTy!__Bf~TIJ;us*cP|C6moa(*_!%Wz?NCpWxx=05>hs^Nd z!V?HD;$`OYhn=)jgFM7A+mWI`Ay|)03e8(xNJn6UMkaY;kg3k*fkV2>XXZci&0D(- z)f3(bTIDbqcE!rij!8~FTM6Pvib^Wr$2_Qo4;6wiE!6F&SC`hIB2N-I<65`KGmkdU><5JVxIRTIQ^BfB))gwV&0tXKrz3+%Dkx-w1*efvqEC5&~U zQXYvh=!6aY;HXP1gX#T!JZ3!P2?vjv52_m7G`d(?hlVOB2gPT}7Gi*6Dcd1bq9H6FCuDr@~ zqRXgmfbu_+m^t=`5_w2^k$$mK4xyctjvtcQaR7 zrk9=TNB~*vQn(`brDzl+sy4o-3xGhu@44o(i`Dwz@+rYcBaxb^1h;`C2TLe7Lhi<$dt$)+x+>lVrR(Bt-vI5}FV?h?j$lyYO?`S0>{Yimj^`~PDme`D?6 zGWkJ&W%94fpFND}8|c%O`%#|yVX)xY6KM=_#hm>vA3`?S#Btu}u_d{zWP<@MIh7F} z$`l}4HMZ*3#L~zL_uKGamgn}`?`Bv?%jR=BFLPRpUTsH=x;W(>?*XI+8U zERe*#7pp;#TUV-&4a?g6^r$jQxN_M4v2E6RLoSSBN+z>VPuph(Ep*=uEBq-LSm1Y- zb=67_#F}4v2XJV@vzSsz$}ALmia#^(KE6KOmMD7f_;v@&0mC0K?-dIxc!N*pH%)5Q z4|eHDd4&qs`t=-Yxz1)T;^5;ZJy?uG-?F-eKmEueWUnnV@|VLE&vZ3k$Ji@U9WowI zmW*7d3f+O?#$5bA0^=O_rp|qb$9F+PZyu)vUh!$rDlH{ts7j`(2Jwh!25+pZDpDwN zVKlV*X$6Wl7=in|TO(2fFM~qRJZ|KRf_ycRtcVW4l?il8Q9T;v&5ZaFknU(S^@mTZ zXhX1w>2EX5+#kZ|#*LXQpoqH{<6Q(wxtb#Q?wn*30a{4~1ECUB-zv|cKpwyf#WufZ z-e$YNyu$lVbff#74+>J?KI_)dD?JY{TRlBy4yy4LT4T5tMVsc?Bl5mq+2{RS1HYtC z=I3T<9Ui61O=a#^H1j5x;zXUa!%54T%#!w<_kUhLGMw(;;P6G-{`5Kxj2*f9My4*( z)bLFlw$bh%itB`oaNAT;PQ^73TtDD~$XXy9$mJ@V)2RHI*M(B|6Ea!iolV-j1q zwm|4~4IW7L>-X$QFl=;R@|}!m!Nfu9gR(3%m!^DWrH15ap}^hJT1s|Y2kT&cFn>$l zBf7Bu?cO40B)n!u@Pb*MOwib0pxz!8MS)i^VfeKj_~KYuQ%%;1QF8FeP+ZjF2m1MA zB|uHxr(y~yrNftR(gDw4d3b3-JbKxs@*b;ViK}~AJlWZv`cbKllvI)Xzf>A8I*cI! zg3#smU_oTuWL){{UD%WSG&ai({dBrXtaIHWv<$fJ%id_ayOx1<3Y_2QEd}k~-G$@I8RXgrR6y?5Mwo?+HwSLV+*O*S66N7_C_{rB_ zpku{PehYCt)c;;A*a~nW05+qQ87{)5EN^tAiZ^gSa(Y2`3-3F4&lpgqlKOh2>Aw^3 z5dtm1L8|f%)ML3@0q5gjfePJ(xY8>IU;&#-YPD+gEHp$-KG`65M& zU50L<^`NH`7!$*7DRp`e+Ozl3={!rzdM3;$e!lwz{U@AA1$Ffa)e!n0{eEXCjesWo z5O7R4O~o7AATmL8!@=J|CQbwHNI=@WINdj?Br0tm#w4y28QyvGeEG{RecnFWg<6OfKo}4xP%lTmYp20VhotMse7TgP;$mC`qQ$HSpd)u zUSFP|PdN;UJb3T*RR!Tln3R(NjbP9Ai-n!WjAWbw?DGn%K+|hdyHF_Pc>wnmUGLAh zzp!AJ4Q@bVVf!%@{y!nYe@%!0=#uJ|f!uEp(7%zU4>(MTi5rvL2f#9<@2Ur>1Of}1 z9)#IHW*0xpc$j7(VIy?Dgh-T52FU{C7t3{dx?V z%FX_T(}#TixQq=fP;LaT4>#f0A@t^#^xx5BTn6{vdf$KiqXw7xb}oq0GOOrw&$+a9 z^i9vhqm5G5$aTWhvTXBOs~;DAJ5hQS{>zk<-)8QD;=F*O0q7O#c8KAAO&>q@2U?g` zEoFQ}KX^LS@*|9seSTqieqn{i83NXX+Tujs=%#qe1Q7GJ3Sn~TAbr|C_ZW}i|xoz5F<}=Plf>gJBH|m|-EI&w@2*_>D3V>c1RkZ6k1%hOB zOo?@iZOgmaw{f;DqU`AUsRg48#+j7oH#egc$v=EXf~oY6?mc_CX&wOG<+l;PK2WmT zqKxhgT|xSn!U@V)$cpG39JLEwqE%Ygwa0{x4^t&YL8)PFea*V4g0E@zILT9M`w>*3 zixqHh?t{CSMzBSdBXXm~b<4DzWmhBv#Q&T^4M~-jZd|Ns2`<^Hn_vMMTCYHmE1K~` z9ib1l&WUmSQc#6)V_E6{Brlt$92B3gGQt31HvjIC!|D%3``vf#(k#2<8mq z4E`y%;zt)T1Lyecm;QwO*oJdA>)mmuvY^&PksSg64;P2vZ(cE-vgR1e3hZUWEqDz@ zeTlou6TFsg(X0>vm7%4-z=pc!wjpJBoGV4a*g7T!={IYBs?T1Iqi(Sl^LGsA@;~Xh zydC#KuY*X&%qpUKW&wii(^X^|h5BMyVld&N94xLA4->Dh72cg>P^>>6D>q3Pp3`P$ zY~=F)3|P1qgi9@qOmD_*WYEWz2SHxu>%QRJnE{r5MG}3-17esR5!bfvFUoPJ$r)ts z)7qK$5F6`J$MnuM@#8KaRT51Y8Y*P@M-tn*%4{=w6X9Aw=<+gb_M3&t``#zo(8#55 zn2*Dl54xeG#atG(mc_D*kw3WlOLfVpIw(&6C%08;_^sws(yOOZP-38GTpUNJa%jH$69^W zD;7+O*%f4*h23@6k)fx1+lAy6vbIQtT#Gd#qFPGtjwH@^Zm%wOjxi57rHN?dZcZfH zuh>>rbG3WA-DucolnW4q|5!J8_eF0Z(|T$|*7#Uet^{ZCJy#BX)WlbZTMb&tF@Ebk zs;ZuCYK$VZqdxd;TP#2Q;HY*-RN69;IxJl?^6>IsucII_}bQu6^vhx^30x zuV`Epl+;Qy;ESyEyp{FHvhW_aMA1DCVIP$&;nN1id(sR<5|U+jP{{-VzfvthHq9S8 ztz}|nwz@AIY_l)*i~BKK6|Ae<+L-KU*>V)G$k0vr?%B(^<4k<)yMFP_Z^wbOc?job zb0Wrzz5cu%DvJ}k)ef^x(l_jkME52AP^`xo-~vk(p;|`RBsZdiMU>|Nxdj3h+FHTw zRE}u8y8SF!rwuJUY=iTbE}puGc{Y4;vasp4tw{vyx4}bxl`*9f$6CuJ^vZL%SDlYo z=bVjq$eDatBf09F_QCzw*K+sx^8WLFl}(1NiMA%KMu&YgvO?zt0<755m}!98Vk9m) zmM)j=iBi~(%+KtlDJw+XlDu=QXTJQnl3s1h@qubTi64csFhK1Pe#H6?iS{N-n5>L> zdf^FLsl@jOU?)$ImLq_xx<;d>tZ}3_4wKDRVi+Hhei(TT43ur!bp^@<(1ygWFE zO&`-r){LW2CQQT?ARi|y0an>WOFzoepJegwa&H%g(2(4GAH;n4JU;GJh=fac%&E4v zqj%pWMZ6K_jJG#s6&wiH5@b?s&y2Tk|1-%+>0MIP+nnRAWxS0VSwejp$UD_{SB10; zM^4+Cj66K-rIi&rED&_(A1g2)7t^)N3NY-fz<3%>Xt@B=Z)HEiyN>Ml=vWs-mY{x73)c(Q_^O3k6jHdm)D(S8@@1YlNHw)jsuI{+M zCv6a$eGcV6offO|Zlmp9)<#>O5BT$R9y**nUS#-gwa_qY{-KQtTx;4GF4sh6f(OLru&A5ZL&z83bNe=1+g>D1Ji<<~NEzjxX3 z4h<4X_6@7+oJscE`xc}xKz*=D`EuAKdzrIogMxSc1_e>2joCX3IkZw{cZz98mzXP2r2gC^?EF6s{6C*Q;5+{F>60e%X*;NtvHg=WUE|c_#4?__ zQ=;5`r%E)dU&v`HiEN#c5N50Yl9Fg27E;$;uT}SC;*(}u+0$w>N0WDEf2?%dPLG@) z7P%tWCo<{ty?1r-H^-E8A?`lVYhq@gj{NDdAARsS_j0F)LK(f(MIjGuGbdd>laeNQ zuFpT{%oMPVzMik;=4owU+KI7kWx;zl*y9m(zf4M!Ge+yLX*Wjq?7SJfX)Jkx{+n5| zcHTVwRxH@P)I6|>rZNY82tc!`pOW8^UJW;B3#z5c-S^98=4Ay?#6$hAwPa8lVNZ~Wyn4J=lT z(QCINs1AH}!Yj@CXf4eqkso(fh4$1;X?`uY`FY$zCyVpI@{Ki#rM> z(!38f-b_id4|`BIT&PtSjtY+TvTCz%4Avo?0wFPyRZa%k(z6Nfg^p9hdXdcoY;AgT{;UhdlNF$d&Z?8U$N;Q?a* z6(3&Q5?$HDlvpI<0j93pu?Yc+5W7A2jVB|{Z1Drt$>>H0>+Qb5s*1!VvwR--MS9OD z42rmon0Ge+eU@qdZ1RKZs!YSDFqflvSSkHwVo%eI_bS>};*zK)*=HiCbM(uZ^sS~i zlOm)80U*wO2s{9CuPV<-&aEF&-(PN4yXM&u@4cfbE*h8gHj!R=-qAmRyAzt2XkMAM zv8XC`E3s;K-ML_g=ItjE*IY+{a@_>s(7dX=N@G2->bG~<;kR!f#YKZ_$T7odu5T}1 zcV2RJm~Z}mgU!muxy>--bN+DF5yMax9p?)7Rv>x<`2L5av%KDu>Teu7D5)`q^D zDhkA}kMT5dKM3j!6Z!cOU(^HvHw|DoW=Yh8Ag0*ffbf$hqIiD0dgXOwZhLG|l)j#_ zEqZdhGP&TE`XR?3PCFeA| zMQJLe3Y*@}nkJ-%i$iLX$3hCwV8ulN`tKtzK-E89B%+9 zbaW{NE7Wuojcql;e0nM9ciub>gHgmruRUsgHu1%HUh+vWEeH1um5nzN?X`HL%ry#k zkecUCTdH<9sZ{o7DUkXd-i6~{XUblycQVA+SH;)2zs@{T+#jn->UXR&#&$Pdv(qGk z|HorXvy$JcJITX-Q|ZeKs*ajQ91y!wR6w#H-34Q6>4k1FoF39iFxxJ|j2{z*ut0%# zreNgN!m&Xij~af>Ov}$kV@+5wAJ{B_(fy)B`cVJz4g-rlSkd0xgFU6W=NFPw_uDkP z*O{=J^++RpedR-0$PI+v&&lJFgb-NfqX1JA_z>dOXt$nm zO?g)3ltBrG$^qXPPhMXIIfO*q;ddr9+LfF3!6z=LE0>B3sV1<(?m;FmabDhSDZN%% z#QsQ>+%@lLJ?WyLghWf7Rw>&tOs^wsv zs&73Mrf;({rF~*Muo+He?!~3{9T~HYJgOe?u<$10KyE+IQd~~utEJnu-Jb5zfF?!6 z5h;qDp)6Jz)9LcBfq|mU_-p~8=u{!ln5u)>X+&uSG^3>B*rpU#)eZ`Ctc#&2Sfdx{ z+2SG;>7tKVqP4+FS3)nO9&VQ8MM+J`6$l7K9YYvsoWRp2egTkQ!VBsF9pji6iP z-EK8jK;6xs1HRT#Oqo;I{;%62+f&`NcB3{C*E`i4fYn-yNGQBT-COnq6EKBJ@`>To zK=nI6`gcN*OfzG^yvTWhk0x9F+)}u0P1v4*-pt90p&X$>+e$FiJK_r_GZfsoJ*r(H z6|y&m{OSHVp=~}bdTyL{FGjtGdG&aY&>xHdN5+g@>Xu)`(_Ny+>j=mh+APlfn;)dS z+2&{+xw*M+nP2v$jDumXvksokYF+J@ZF($su7h>%UhDqr`Zit3-Id!@9kyQ5+eie} zHsk>QP8fcFMYq+J)H=mGpDM~bf)9Pucw2hz7gvX>rm#??t2y$-8EX+g(~`rX+^zNE^q-sv%wb`<1@Tb zEwH>KqdHBmH{aFhsyIq4aCE%9g#_yRyWy`hl|%11{f57DExxDrb*AI1D$TVd&=VXC z?rvHR-)M0cxzT=0>lVLN_~gN>+fzXsp}UFrtIHjO$KsY0071-(J=8A)LHC2iox!jJ z7`wX3n5FqkD4C`ZcO^BaS?lhxlUD;`n#M&NX_*mP3*Y8%G0uIh5RMIN&pI39*RX{V zB;US`m?5I->9%9>DNoUp67CQ&CnO5bEX$OJ`Ho+;=@m)+rw%0`Mz=mZI^^MCJruB~ zSVcC?g6>u9A7ZmS9wcwBxwYkOF3fuf=00cyA3U%I<#tGUHA0UA@unq4q_-9DTZS zUm3?SB8)+auzG7V&bt4ysuGIl3{bGF<3umN6EnBN?wPs|MqC z^t`PN$0&(s+jApIQ>-$(0LiCD%=j`3c|Y{d!~XPRsmrReto^c@rTwyhIaWC4XkBP( zO}n9Q^DenJcYEq7mEHaui4V`!LoD$1HTb>6ZmZu?o2jp}nkPcDd`4v+A7fzw&MdYh zP8K%3Go0oliQR1r6v0OlPPtET>eJoB6vFj2ASHMK`jY1#B zZcDHZ(_}1Rfqu5W@BEsI#FIXrt(MQIvw*5KM6FS3e7{ru=X#r7CS$>trmjN{6`Ixvj0pBSavHE(GyG%12%SDRg}Z})1y~a zhr_k(CaRui2KtZ#EeG?3!$I~jPaBjdOj6M*V6!TR;bn9aDl+XWLRYVlwDmQ&4XnBI zUCM)MoMpP(uhW%l@n)ic;C47)M5v}dZ}@%3=)#J!Uy@Vqt}IBJ#9;)voSqu(vWTL9 z!`ySYie5U(9Sq`2R6I#e!wrcNDyNBvStA;Y(68`Q`{uRH%5qD#dOTtpsK!bR6=9u# z*RO;Mj0J&h5nOdx$sue?E><1RXgyagnjW(Ub+Dw*Bet`OC^ki{IO^uRJ=J!Dve#am zGULdoL*&uzcVETcM4ZU0!fhs1?K$+wespW;QQ+clP4nH;s^b@aFJ)Bmb=K&zut!0o zGRrJ2WNbe%d+gE*BgJX(%UR-N;ie&ik;KCOAKiG`i?rVgYk;+*$g%aG>zZ7d!0Cn- z0tN;|1Vc*i(Gnua+B(j!k2bBV%zXCl=iz&%3rR;tfAa$$;;!yS5)cdcJkwW_+*d7o z6A4&fr=2*E&Xr?tAORmtx}vYNVt0$~!B3wcNBQL9X?84D%x?sL_y}bKJ-%(}-JK$i zRqbuhPp`9V>E>`VTdMTLm$1`zc-T&Q$HtzfU9Bpb zuuqTfLz~t4Y?__nCEgiH(jrqlS_m+=5()5r9Jy3qV=%^FW070{|$QZ!mX? zfcrOf{EfKYkB-wngn<>y8`p&Oee9?{TGD@WjzlONY+G>8;Yi?!@R0hV?Pjwt?_RYy zWbH#KxV^?tj~>iJtertonGyP8?b`Jr?>203>RN8!b@AcFt?SpXudY6R=4!RGX_qwR zS{r5b$=Rz*FYKDJ>*%~bOxr^`FZsk=lzAhBJ??HWrk`F9Yx5#P;^?X9QJk)wP0_YR zUmDB@bxP`j?v&(4#es0f^WTkT8qVBkuvB_YZN!|}8p`Kw&&-D`JSn-xnDp!0OPAke zF7+%vp`m;Qc6>8r;aW+#UrDbz7E2@{N?t0<*pXJfm6kb3oO{+r*Yhe~XCX(cJjZjF z)HIxowd89n%XAAaPOm&kAWKnfQ!CoT*n7Wp9zYR)I&IyMPkYC7v3|j0Xezt6cV9ti zJ#MMZ&~&XRe0+fAVw_z_acaf|u22|g`rRqN;(Znj94VGYeYCp;YBzOljO&hR5X93) zYZ0(Q0}?Ha@tB~7cI3D2l~?Y--X<0QwBR;pXUf~di`13xBU4Ov5syKVn@+DuevHdN zp|Jg{E!MV&^4h}c5C8EOcTw2htsOR&SfA9$X@m=J%*9tc=N=Hybhw{Imx6YNGc>Om z%{1r^^;!Js0!K67X&gNSv1c*m>eX>O3sz6$ON+1k`4?MRay3>jbJPslpP$ZH!v|<$(WkaQ3-|`T+%Yd4x)w z@u7OKN(QKVxTj}e5@MDd-TN{)=F5m`PMDq6&(>$w}_*oOCKI&%1g%5Af6m4C5A33A@ zqG6xa()_pkqrg*YgyO#T&6{Y57i>}D!dFkfykOtv?X1m11=bZZ604Ky-XD&-XKMx3 zt}|N5QRwk}`l~1$DQT=MFPLxAT2bXrPY@}%_IOnJiYwX*7$1Ah`ak|6 zu{|DypN4u0%=y$bw0PzUQTV~O66^yo_5rGiSFcubb8i?}zm(sS6tohoXT-X`izTP6 zm*&6Rk6N`;MgMpFrDAD0bK>O-HaiON%J;Q*B;XczWqY1Bv$^Pg`WQs|qATNRRQ46} z%eN@4^Ub5{K0bL``{<<;zF+R7PMe8?LX)7JXV6>zFu3W9p`&BX5-vFaz1mfS}qoeS-D(2!kVk67{$fjAgC z_}kRg9j?k=nda(|;;WYxmvj2>GHz*=H*S(!@Igxc$G-eL`vB$68!xZ#yed(8`SO_K zj2UJiwaWZTy!heL<+`k;o)1nix6X#0kun2`Sjw0<^Yx|Rw^`eKOG;=>+s-TuZp@m) zoE!W;b64x#C9qGQTp70X@-*aK!yHjey_Nlc@$7$NJR6)U;taZFUd+Y`l$0Bok5v3ZBdK|H7)1N92QI&cLfASu3hM7I=r@vF~O{0qcm_GxiWVx(!RPorXN z61O8pOLEoFqzp6CV>(pNHMi#|OMe~~%BNvT$-=>Vo>Z7mjjw=(;p3U5iopL(8$3(8&}m-Zi^P z3+C6e5?NNEYA)g5S5ha>?^LhrDzm2zkU7REQZzj0RJaA%UFgX>n{JkDnQks=%emnR z_}(N0WGojMB8a6B#-FPuX$#w=8+d7|S~4@_&y_o;eyrC&m(G7tQxn3xSiW90(|r8i zcyZDrf}*1btg}NWw90|o7(F4Ukkt*W&V;wZQDq?met2*Ok7n%XFlkbnOgAR_waX_0 zOI9W2V^6oi&M_28d!&_(702ltgz@tHGb5ArvQ#Ll{aV(U&R0xH#n|gb>_U2qPPnSp zBGD5Jr2M@n_^PfTINzQJoR8D8s5q>ddjJWZuUm^H6rPNk-P!L2k!S~0qs0r!Z;%E?p0*780n!HS!1y&Ia0~hTj#f{SnRaR<`NBK{b=kn~y-$(jck{2=^2`ZeWtLTNsOEL zt=X+*h-A|~I-WITm~CQd%gAI_Sr)o%6KV^FTIwhm_5vC|2^V89)?`t?DX3!3!c_^F zJf;V~hN+!sxWvCUG;U&3(Rm3Sie|&dj{S4*{4JX5By9kRlymM(trXhEI2w5iI^MJR z5;YNv!_-FQZne%jo5y)&u}G^1I@VM@k0?JeYKce4FbA0y5wM%J+zzBOI){5Q+Ewmm zt-%1#6O4)`*oOzw{_8u3qhJs_py2Rt)- z$SK+-Rg_yEqg;BxZ;2dHpP~?yUmOYKoo)8?8u@;eVfR@6|W zldjF<35Gwa7HdA>aW_PJwREeDW1wAT5g}Rxg7CP%On%wbbLo}!;)mwr!Wew|!&Wsx zVN%Qu?{-=zjV=(l*I5>g%8SC0B++f{@4HdX`qh zZ-e>B@nTyc^OEX5RbRmg8lX^(gJs5JPy$-31pM}0mkh@|hq5`e7>PJdUo;jX7c1zb zlOY^#qLEA#7>$l_Mw2C_FN-#X*Z0^I{uGl4i^RwMU{{Owg^T?Rp`pA!)U}XhSl3@q z*aw}US=8M%xtkzL!)9h?i8+KZdR;^jGu^rO1u0xFiH$pVs-K0TU6xK85E7*8YW1M{ zAf2rZ?293hu9>LXjpY}8I0I_h)#=aw-dpEs0jQx~bGo7#E5>s4O8|rv^#N2;sFH#9 zo8zrx8DMevToZOQB_#>RE)moG(ga-~C7I46bUZCS#qH0vam&@=Pev5oGlvX8Sn0m4 zFGY8!`r_~1-B(3SnRE65kX#Vw@J0sbu%c+e)Gin1p;56|(FP7OV6Ow7XK8=>l_a4~? z$Ea+LSZL}g4?L=N4-(A}$AgVMD>fA;u|Y$KPNVUhy)lRsDxda-7xnTS z@bgRY)n_#6>TQJRMQ@rc)u_P_bT4ELI^5NXy^RM}a;+h(LED=;ao6!4n+D&{U=AJ* zzY!CPS2!{FW*2kN^xBQsy;E1;nlWpdtLuwl$fO+~q~LAa(RJmV=2C?j(mxmg+wJoS z#s9H@FTmTD%Pm+WVq9EK?|`pb`Znud=Ze=cw_bsr-fRY7?KOLdzrLLl{4Q&)_k{<{ zt>7tTLz{jc1PrKnY-*PjLRU@vU?zD^BrWwG`Eq^cx(}-)l93^t#fwrAvmPnSKl_aYfQEs7Ww_MkX$(*r#7xeuA@@*t zJ6Q_q2=+(Ujh%2&>2m)^@#%;DnLJV$2wpo2RrsF;)ORHge+bat?LRCP$Mp$~^Q_>k zDN_(ZY58+C{zHB64FLV_F9{(y*S}Wnk%d<-*49YDEiTUXJ!WQ8>VEt*M0%&|#C362 zwy|`dc(qOc%aW?BP2&9M@;`JD;N!*H9^FZL)^3KyBS^1$u<1y@u}wzrGxxQ5ho>U- zhRwpBAPzknlC>m_4@OH_5jP!ff#C5&TrA2HVx-(GV0%;$TnIqCDPtYJ|gduHj%?)bhqQecq#1wz}U8|f{X?R}1F96H9S}6ttv(7^gn$J@LoKWr87RFK8v&nCtqO@kW zbe+(2T}HU@faUV9=RJmZ#a%J1uxDY($d|W#bP_jF1G^_De+)V?1 zWOZFn1STFg8CsE%k>UCF0@2tc``Aqbbkjy;{S)U?yjE@LyCSfQ^*z9@d0?XyMb1Z6 zaAJMXz9_nXt<)R#JSY#1s&U4qQj2;Sf2O|DK2cwZdh|T!8jt@H$NDOyNdIH2y1lQQ zBM~^l%VIT1tbXI;Dou$iA`567ie2>4bh}uU7b6NZfC`!uQN13dReZe)aI6O0++C2b z{r;$&_F7RY3;CP8*4ME@MfSXPn9(HKTf-$5+HIcp%`TZWq(bd-)ju&;f+jVZ5tQ5c z(ow`Q8WC-7tWvSPC+JF8QO7P~$2M0}=B@|3%=dxjcgVI+r71bK&yMKkV{$ll=@IRM z@SFq!XH2^lVb62RAHtcVsgG$-k}ErGy`*3j|uL$yf@0?LU2E8rreaLxBX3fF0uW9j*D6zi&D;y^k7Su*caz4tb@Y1xt?P+ z{~!6LuVQU9TV*M5VWYb-K{)B3gqdWkCioPL(dUiXHp)`40TW6j^(828Zc??%K1AS1 z7FdxlYP2NwA>zEyd81WIb!rtwp5OO-KhP?No?}Dm^x6eQg2f|Ab$;^v{dogvdh^Cw z9V1{Qd3{%cCpFe*JPxWCNRft{K=Zo@Bu>#}qGg=#L2h%SP`z+MIXxY$1PfGMJIxEj z!ZYjB(K>}asqXg}na=~&vuSvwiwmO}-JR>V`DxG1)H;^7t+oWQbxXR*PLfyU8zPkz z6W2COPth!d#!j%+AyjzgJY9P;`Msynd|To~y|We~ndWl4mq_o^!C2Rfn`*|UzUOfu z5UNoUOQ*UlI&`sH_%nlaD>7NEi9`q9;TwFpEdFho2$ZC@gZy7Wz=9IMT-;LHSI*&4 zA?)j(oDjS5^Hf4tT}WMUt&YMJp=6TNj&j!JT!-QqdD;YqEA1@S|5sh1Jr$3Iof-Z} z{pDlIoc|^r4^9t56+D?vNjYXE0?%>o?qr7Jp~CkjD}Y70h~myYPvDR1B_tU*k|$$? zD*1zVanFb(uFrKsGT|M+gr)@NV?E1OLZx}_x$;Gum77jhdsbdsSf?Xz;5^bQ=MVNs z37OzO!ZN|`&;TyNs&o%ii_%p?Az*x z0GAJ@$6-QY+jZ~uiB4qk#X0=^syN(HTNERd%&xh-P86PFzejimP{c{pjb*DU-I;#S^}j19@VHG0hvBHp1zFAYD`q^eC$b;4Maa)IsBB{Yv4&I#C_Icrmql=FBJ{Q{M4A4(QJpCnDGN z{aVszH9xOoTNFUHx8248wYtOuxSPmC`~Q*83q zE}F(&GIgPNNTlsQge+4XiAbRS>mju_S?BQ?Ds%qMlmQI#*Ru?u2N28P!It_K9i7YO znIWrOV}MjeGlHzYgHFmANS2M?rt z_6|uIp!`Wv6bjmdD1n_JaVD1V!1~wWLw`_(_eP6VVHH0R2-=zb5|rhZ^SF(e$zQo# z?IIMRqV=tIIn*%@y4N4#7kA-Z5_GJR*+$pO45p$LIRBo2zYsoOv>KDzrRCvTcTk{1 zwE&b|7yvwhhI(_v%C}i5KBRbBQ`VUYinDclLFF=*%fXyjd*|kG|3mTpO}AbAE9$c# zlbiN5WG<-p#y&>o3*zNq!~-GOn1U%%N7#I3?&r?fZ>DpL6jW35PA#tkNh_DeJg*IE zmICS!UDYMf&l~kahiGp)C-`fXvk4v*oG3u%3Dv2~(!nM@x5sm!KZZHz6v-Q_(;%mX zq`%J$InwKG9#vbD?wsC`>AbraZ4g!4_SD4aZKjFl9uJkM@bU~Lqj#A~8~6CgePXzv zdFA1sM;i7YJq{YLWc3#eyfkSwq6ibKm}vn1gIYFN%S@YSgB|>?h(^4qLOv2rH)7Fu zNuc&OyXDK)0U3h0bgR)8qR#XM!AD+YQQYY}INaOpLiM(@6|w^|=R}>6cMJV3!NO6M zKPuUXho@2qKR*Q$zpT^m9%d2SEljY&HpNBZY*jcffIG}Cf^hQI*uy+*o*v)zL?D#5 zaVgrs=k`X|J#S1i269I`s?Nl}M=LK(e3Z}Blz-dN_HZ9fM<1Le0aUuv}iKViHlXmAUQP8zICd` z5^bF))N#^+UZ4VX60P=EBZtJX1gPX*2G(YB)92dy4{6e#RUb|9P!NR9nz;OVUSX$^ zxHx&t6pujhg#*4P&pkuCC16*L7*)}ZUz!ba&e7AqT#j7zR-+;DJ{XGmT>AV)O_ZAa zM};tmGK<_p(}!;gnuvkAvOJ#QJW{%r8>YJ;ll0_BV{0(wf>ybEPTM%`QgD&)nR(-r z1-{&?Kzn5nI5H@aGEl*u0o;_Sws#ZO8%OQx~d!?--XwlgJ*k;%$AYk0hN0BQ6F z0?8443_kPNz%H&uyMO%TiC{UL_Xoj)0WQlEoLzCFXhb#6W^61;zel3Z&Y~Gn!*-tR z(&p!v$N4o(l2UWp^Rngr!~gwysN&bNFZbKhf#7Yl`rGqBryV8elMwD0n&_)oG~X?s zpUo5;BF8?-FOU%C6T%qLuGvy&faw{y@T zH9322Tno)mujqWV(2c3L7&lq#Lx@Fl;AHs#eh56p7$72~j)DI0@!%uEp7X^%TfqK= zgToM+75p(CvN;g+SZNYu>M@0BL%EkG{iTvm@Xs|(@lXqRzck^pN1#YlRuyb zqiPsJuDh^~pgw7NidV}I92-PHREM$$hJ)abwNm27FC04M(h4u|J*&%oL}_q&la6K- z(lM5K{PC7~=}ifQ2ZAdRg%1fCXhw$@H5Ox@*Bg~Q51K5r=hTkGP4b{gm3PQbFdDuA zRR(;$V(0`;uNcEsk259nhpSdY(~DM<*)$qkPFPwD8*9vF1da(dO)6k_oEph%;Znzo z1UlS*8{r=}_(_z$#$hhe|S}-Q#_p3J@F!1>ht^E8io^ZN7)X#`=6U#0^9ATs!XaUr(Z6*Y zu$AKWw@w6qKM`UUBwekX6y%r8yAzJ+_Rdlwb@}{3ivWA)845$ z#hn&E0K}FP3=P-z7v1wwci*e1-^Dw|n1nq0IW_+tnIH4e&E zc=i<|-7G}QTI0jdf^eTAFIU2|2CCgy|E;+`CxrE31Y-owtKS-vgc_|^6zmqJF&EB* z!duEY&kLXC*$BK_T;raE(e>zA)g6{;d0Z+jo{I4;PbYfY^Fk#T2WI**j-g5wye&jy z1DaG{Ct)occ#n;}b)UHTISRhyEfs5+8m1X${6?J4%ggJ}V5BKZSJIC=O?d$XDcFr) z%W??WI4fL9NWCUfC@(IuYb@u?681&X-eH?PM)an(&zgY3Y@w9F_@p^R2h=C7&Xdnq z6M=qHi?%VC5HQjTVrP67fHY3=0yxp};rd#x;~h0~EmFbD2SWz;nD+#213As-lCyjn zU7FGFgR$z3pF;5Ht?_-LEQ`#n(Xt59YWJ;)Ba{+x4dr>t_CNkzOpC>rYz7uNjqAwZ1)^H<`vvyCpoaxN+A!h!Uj# z239UW2z2-k+*%*Yw27udf-lV^d`;`)LX-f26}Q}SS$F+~Re?q60y?Wi$aD;%Vn#{w zNhU{aedEZFx9}@TciY=&>v4ozrkZAE$(9Evv-W$w zI-4#l^NWu?PXu8$Q*L0NSSYrkD~4#z&X$V>)iWCfCy(THU5iVE>UZ4N6z-lWO=;Xe zadB#I9mk2Mkq*fKQ0SZLVL-wIrDyt9JKtl%;enPm4>k*K$F9z5Iu)JnWzQqwba6Q5 zr2k^hQMCX1z;TZ_J4MI9#)*E(c0p}U+8BG7AJkaDO(@2XOS|)q=u%P+(fJfjGJ#u4 z*t<}2t4shipn^l5EF@c)umiar6Vd3vL%Q^MY61`y`_&9QQGKGlQN0jFzBn+t%>|=}?^`J`MynVJaO#eYw2?Jd866FcL<<7%R4wwMUE&iS$s&Gt6ctb$P@LEF2xde9{yA`?*ocldeoDR-(QU+q~ zl4!bilfx}0Fix}a;Pp!e|Do+J9w3aA5tFJ_h1`w-E>w#g5W>7!0>`aV&O?-Nn?W>t zfN7jy(DG}A1VK61VqE(Jd6IlpkvGvH<3^x_GuzpPW9?9L(F_o(Rfyt6QC&mfhMk{J zMZ4f97JXQ}`_%XWE^x_-*%JpzYu6@fqnkagU&sva4u)480pS}HU65Z+U0xw<48+H4m_g8BQH(8>U0*A+h`W$oxxqS;nI1!vOCX)=$8%xt`ba@40(c*a3 z&yFO$ALsXXPf2>e%{Veh=Evj1w2T$ygw!CK9j1gu3qoKzW+Vxo2J^jjyE~w=JiF%^ zlifZ-iZy)fHF!97i@RU5rweH{G_D`xBO0OvG{6B-Z-L;s1QsHqw5uOR3UtI!T2?$^u3aH zp6wk2ox(oQAnyDC=To&!Jomc0{e&0qj5wgj3 z=6b_1-owI(`A;x(Juu=4FpL@wg#3q&PC~qz3Q*z!8v!QM>(gB8lq1e}5`yhanv?zE zUCOnUtS3uF-yt%ZSUitsiF8tjUwC;5%SL4+{YEw`tarR(73R^Q8n zw{emM1DGf`aLgzR75ID?I#^at%eUr<0M<`07^|HuKr_0^kd496^x$I7&d5=_Ohaw)oL9 zL%<3HGlL(w)Bukg0Y0=vlN3C}cf(sT2U!jDj3Pg(Zz=+u3LD?3f|=^pKIQq(o?&8Q z-A>ia)U<&Z#(XzZuvbSmn?;CZET9AoNkU?6~AKz#OT3? zRQuD-sd63=X$bX~4~31+&4rshfSoS&VPj3BQ*(_Ehcx`X=cL*J?unGCYd3un6NTQhylP)8Co*t-EI6-fJ+UpaT z(K%l6?6?B$Cj9z_s9CW(*v5beoY6#Wz zL+-58&AA&qbk{+srHz%6&drrun-mifS}+9s5fVXih`ooia*0JB_%zAq!+m!4+?OBj z?R~LaR#)u1C)pFu0MzRtZsrOLV{!p+fYyo)c#5NTFdi;6xS=I#y();-S{yVWoh=jz zWA8od>!B7#vVojXa$t{|V#2I(=w{gX)s?h2ndY$!b#kes2*e90bF0};a%0^aXzJ6k zEFLbjBMoatRm&CYepR56`m;gqOI>r(P|z8vM9TwzPxL(Kijo6d^z zPOr4B-Al~v^BQ{|<(+9VY(!_mhyTV7I9ja!YU3H5LitXf^JB{LnZEZJ&QqBfmbn&XU0G zVcyLtI#Et+7_a$&kH%+YlsQk{p!Pm^dDkS{xRHFOO<&(5sX%3hS>r}^Xk4QQ+B=_+ z*}xQd_L7Or7)*^^U#Mo_+6NOX4+zyVy(uOYpLrk(N{FZOkeV&(^`2nzox*0u$j3HN zw@SKi*LtB-#Vt4$h|v~1X|j%tPkuQ#E!{1~zWv<;XrqrcH1BNMWGN~oGE(R8B=v=F znMtYv$h@&yJyJ4!tcppB6L1X1n!Q_w>Z}kk5uz^3b7UPxpF(vxKmhPBHoQ_i)It<@7oLPiOqH?c!3E5)LM0aOc;bhr# z!Ra@#(8df*Mg4JVzCvr&$Y%liaM9#2(=vTB6KF@!{;?jq>=H>}UF8y~Vs&}fvC5z&Xq?)yCM zuxnU~AX4l26jJz)8wV1>&zh-u<58$IXF0M*m0led%`7Qq>kYl|K+(4#mY{{e&q5?c zG;H>x1}BM}Y9wMkP}XN@WzKfgki|x#2|(%aQ`)DVAhmTv`?oS>?l|~_$0L2LL%U3w zGM7%~gg-#$px&@tm@&!6*>>qjcGR1mP-aYsoi;ofsh!`AyMxO;*A@ty%7N^tv(qH_ zw<+QrxMzxH*bQp0XO8m{DB*!-5W;f<0Y$I*!|~F zJJl-Hh=Blvrq~AHx9eaQY(=$Sz(|m`2_U1v#K&a2qXE^|P8~BxG&qj&>ZL|H>q7HW z2U1wJ5pQ^bcBF|GH&;X&wfS%_+as!$j%FM9L+Hxeg@i^=fwm0p0o(j|| zRKxPKav%Lc)eLGiPi^}0;&_S{r?dG%0P|I*D1rvgOn>=aFd0|!!zSaV5bYleQA`b2 ziB`b>neG0sG28tiGqe9^PWiveobs7k(JEjIFCKP^xbPjEkA9Go#Gf1^zcaG_=cZz6 z#2V-xMuDr`7QmJ)!_MZ4e6?r6e$<5ZKa)p>hxC9Q9jK!I!`)j{hN{c_hQ2&VWzCQ(6@E%MDBf1NKg7kp?a`5q_ zdY;XUy32AyX+WY^sbB)ks-v}a9^%aUvprnQTF$yV9fm~Tbfv$D%0`e&U!$}hG!Lt3 zx;A3o#}DW@y&K0)!H7d}>Dn)sgR`5(6Iev1*xT=)GNn5Gj}b&*c<~7UPydKFi%P(s z%*wWrW=56g{rq?4>46>XU!L)h|7{h#Q&jmUY5ph2Ds9yRHoY4Z>N^~b7o%p$xprePsbkw zH_S+%A#Yo8O?1S?*Q)vdEQR^#zNcG*-R1pu25#jCw@t#+>Lvli^{XIIK%y zWE0&ZUx$i|4KujVr?);W|J}+%U8`Y?!$yXt@k;no^}MdtX&&$Q%`>>lc*nM)Tv@vF zUo&QZ>7R=$&eWy58IPA*rr%k=Wy{U$FKSafyw)s`O!iGpax|xnwf0l;J+uyEPD^#Rgja^!V zI&nVZ-d&#O-9tE^BQ2bSF>Z z*G=91qDGzmgjsQ=oW9AFv%k$>d;VWDzw}S>f*mwV?T!9j$&JPtb~o>dKVH-=y%E)- z7dJ!KsF`~Ya}DJN-ZoQT==t=o^Kxpul~}#1I-!yhke#uVFMefe|LrS_yER!KPe1f> z*}7@&ZhrTM8H)UVJRj%8&^)Byb>sKa-!g0Kk>0Am6qcr_}cf3MLjz5RnJP~7qFccd6^zDiS_KbB}KOj-Ll}z8-3}{wa>&$Sb_Oz zud78$33(o>`|b?5U2iN8yLp6OWN*sUP*E{&w;RRDKv2h`a!wCshFwZo@JPPXE`K!7 z^O9-1z(0InxI~4-$r~NlRrWR&tXDeKc?GF8_94wc374(pF|oKwmbf?DL+9tTm6E2W z3T2Rk%{1!~-p@aq>hlZ2j_4daf3LTrW5?+=4A1j-OUS>Pdib7q(Ngu&pRhZzLN$08 zr;FFgx~frqdFN)z>M9R|+m0HeN;?*8aGgiD@w!a_Mn zJH32c+9L7eE!{JE|4G#~s~soTyx-M*KlWKyWLxl!Hb$sJ{$M!lJ~!wpGU0`hvEHn* zS7EH;Qc@UdvqHekQjLIy&PP{$FB|Fsw1=syudEsz0TO#!PjT**sL%5w2y-fs~U0~%G7P7-Hzb? zE^R@X*-)ef{B5(VJnqG&==lQ z?fdDwn;3EDk!7#ClB zcf}z3fj-sH%KK>rwH&=^liO{_s$>3~`<{YM|4gZyk8dsE#k8i&E#L6d?&;FW&JMUJ z63rzVho(DjEKe$RI2k~zb95LjjXcNWoJBV9?Cc*V1>rFJ(hA4tMnVvm-$d$_h<5r4 z-=QZ13C5id(F-FB_&$-(TtmpNowlpPZ#-OIye_7<`2C%gnP*Bh?%BWIHwoTSK2{;0 zvbxf{Nj85cYu*ad-m}=--gB>Ak{$@Unk3EIzlSsX3X+pja$p5xQSdq6fwlNf))Ft* z$COI;wf)Y)Nxb)Ji!H+5FE_rhzg+z`HDGh~l0=^!RZARA%_hHo^(&Ms{l1z+HOY8b z=J>8Fce^XsVd!A)XwP=f*GeeCjckdkqCgDIPjJ>~ZM=~-d2}Lp`0&lLgRdAt$2RP6 zxbcpagDBf7qvpjwZ)kh`sJ>xqmT3ra*mut}82&|5mf<$Fu%Fp`k%-NPvZnfmR#3-$ z)t#no+BLPyuOhtrgYj-B*Q{wA{6$l=r*FUxoXKf_ujZBGU|hZA^HbG^gLsD63}(S- zqCDptU-9#_ajQWTx1<#;o^ztM$YG6^FJE3GTKR>ugcnCwItC zzu4c^)#cEiwcwd=29ik zfOMpa4Uk?VH6Xo4dM6^iM!K}1v_NQ5LLh-~?%;mj^S;j@`#aA!_8H^++-!5_&Q;bm z*IaY1i*ZgIdSp7vApVgh82E`F#lNMjN3pz)&bg}A_x1YFy}rK9P&KJ?+!vRJk}M;9 z^%m55_@@@Moa@np{W6=KDD2a@#1U_=fRm@rc8+FchWp_feZDjl2nELJgO2_a#+D#jwE# z_&Or!q)j zGCr1f1OLU%VBioAxD~cAABwye-(ql~wrbC)9bT|siST;Al|o@P-v7A#$OH%vv6p=* z=gN=DA6z^MULOCgOr8+9Y?l+S&b84BfCNP3!$hyx8RlvH*#(h?t&W#VpzRhdG?;El z2e&cVS1||*^y&GJadV`V#9I3Vw%(h3^uz9SaBg9W*r&VZNi1J~7_tQC+Q*6IFqtP= zJp8HjC^(lpOswFFc~a)r=!Nzxa2~3~8j`P9wE+yk+K#M=yg*JAXr7BD92i3t0aSBs zN?~R;qqXAMr)PB4E?kV2!IO~9@0D8oSrBgbm03omCF53;c(4tyd3C7-$iZ%2DCV@E z&X;X?TkoYzWI#i`-Xrc$311m{5gJqlnx!$ES-q3`(K^U`3B{V0QQlV)({lVqJB2o5 z9Iw5JhWRq1iCD&Gpo~_1QL(`7JcVOmf7?z zWMZqAeaO{0^7GonknSncsjDvtfVe?oH}3Vxxbqy-m6wFI+NT+F(}|V%am@6i?A7rC zF}Y0AMieGK4kOHg>x;BKRHYWHZ1_#cv;UfgN5n#UFcaZd19^nTD_bsN0PofDj#51h zjG)5bjiA+EBY2DS1Mx4sNjO3le<*~BQW8I;ue>n$a-4SYoei8Nx+8AO5$0IzKIL3E z>EBF*@r~aT(eohvKC$Tv3Ug=I%B)fELspi;!&z|vWOLH*mAGghe5U^U@ml5W(`Am+hWmT56bn1cLroHvW!KiRg0^ZpjuD}u zu@W1>;6iB0?G30YC!I(Ug;$Dfam|dTuqj_{R!or@ObvuvwGi_-c=F}&(19rjuQO$& zU*|?+>*Yix_Ttwoo&Y4TnE!#7#=9Ba#)PRY*rZikL3WwPTJ?iQ`87-AcIK|<1PG0I z;x{g4Gdt_jMJ_sZryAiBlp^`^VG|bXx{t?gJRca(SHGSzveHo8F5BGi;R{@7RyDaE z)x%}R2g74SeRuacwvZ4X2hQ)nzf+NP*!1?)Lz+(etjCx4ce5yid82{6aYIDNLiEO> zUw*DfNrvMvxNiMP5(RtOLXTNGjOseSsaBKe_B&t9+E|P)Dvp<30!g=7A)D-oq-0Z4 zs+1_~$!L+ctVH{XX6k1=e7CU4(6|6L6Y2UY>sq|M4+-7n-Xp%cCG(XdC;tk(+0JRPXMWOE+htVh{hAVG%a}f%9F%YE)`!C4BePmn7i1TU znV*$khw2l1H2;ffRcKSzNGDN?9OGv){S{Gb_HfaoZL=j%0UXr6{_h!ZrjXv|Gmx~A4 zNOX(5qA$^j8}@@FpyRq~23-%eb5-N2i{E0On*J=||0p6m2_i2dMFmY(fQ@TiUQ(#C z4S~jU>z|Kr6?Y2vEQV98}8^N}J+@#h+R+-=b^Wc(ZKzc8C zTiyAUWT&RM2mF`r79OZ^0|$3|L|R_;VS{bvj(1t;%;8{3#rA%e5^ivmPPxaUKaphr zVrR`5B5JlxdGAWT>wZXD>Yz!kp9j{nSyH6CBY-p1=``)7J9+hgya z7*MN7LVR{*Yn!@NL978se#YWJ>&)RkZV$NV?D8WApA6?rT=vK#2Q+4yibJjQun%SR zr2TV-MOK;nqZn_g!QJL`b34qf%58eCmiG1^?N9p$WR7M8cNQLOwSghl1=gZK2AddT zUGvCVKsUgwCPDR8Ak`Z{K8d>^;e?*6U6)&?W= zqe;y&;hAWT!Lm@9&luabi z8;#^}L4I{Vm#$+J65^-W;nZbx(XvH-WueNb&@l7v=i@z-7q zsa6v(YFA3FklU@1A=<4cEL;%`LMnPkGxeudQpK&R;&@(0AQ+yTH>AQNx$bkXUHty* z&dgI}ism&Zl{B_&rTB!k$?7ii6-lX$sqOi+`W59S-~EMTCs<728?4P8>s*j%$u^A$ z`vuX7P2EhvCJ~PhdKzOk*J0eyns#ix4@51F@s}|~1w#5X7c{$kaC)p7{>RG3Px`t) z(Dd@tk{?9`5NbRmm9724!{JzxN|uSSvsCc@RBwgU3G`#Ouka)(lNr*C{yrtJPC`o% zg<@z>l4YkQ#+N0H#>Kj(R3ZAd;QTvH7ySMLTDvgL&pKrDUmS>$DxM^ClL75xH`yUuorB{5s&(7g#+ z4*m&6*{|Xcah8j#`7tSRJPtMPLI3InKq2`nDvsNyWOWK1A;x zc!QiO$I-kXXZw1wj7nP+8@6if^CtT@MfSOFHHau8a5zS zuG3hxUwyw8%#>w+X>LT&d|uh~lSuKTVr+Zgd=C^9OC2PMG=X@uJNmT>6+eB^!_|da zW7}A^-}rp>O}0HU4g0=%7+4k-KcUtVV;Z6;LZEsA1giL!u;p#we4kyGqlAFm)BMfl zljRw{iRP0Dmb*4xh0AWhBg$9)`GNuPFw_uG8YjXTHRtEN&uVRNmwvRX5?-XT85G-9+x6BWkrrsku=u@WoA!fh@s#1w3RIZp2mE!$wvP+lC$eJW94f;UNQL^-WTBn1uR*F(uDA zt-d7pEF-X~_<%awK|b}^)as!|ezfCA-$gFwjlSiJgKX`S;fM5|I^eTORTYj;lDW)$ z_wzybfpt`mwt|}W7fO|T72KdGEvdGv*sDPZ9S{Manv}F#I2L}+1PMdaJ%wCXBYPa*u8Lu=0kNt|ywsCXM6=e~z)IcHuimME{Ju z#MpzTmG^wsiSf@lay+nM2dTNuzB9!$>2lVpYSq)npnUnJK_Joi#X^W*oFKs3EeC>RU(iSF!N>A`WRGj(Oub)C)54#7=RMtjqkm8J?o60`D9)8^V zO~&uZ=`&*+hN=G0DH|KNLB?;+Aqad#^q(xIaq6Ib8T}Sv(PEEXUYv4RH64y-Xi-sM zB`14WF%>}BK->crr5Wv*nm>LG!&rm#z8pngYJ=y{xd1O#>z?^P~Yf704qL{#*dk_U??QVi;T@$$=>JK4)pd)DyBLiuSs2)O4@zx-PLJchsny}NdoDv#$}8B(g?y8kZqsy5#HLv%mi?} z+)A-xOH;YVZLi=hiFm-#6I9mZM*d;oI&dFx<0W6myn|=c!QaN}n3>n?H@{vVokO`Q zzmcmPbll=gwj>p{D^N<8_QO^nJjSi`JGBqf;#HIV6CS*Q46Fp_gLzqnVmbQa29a@_ zG`s^k!v}$STTi$3w(uxjLF?5Q<%Y9`(`@RJ2=(2KncUcX-Khh%cANw-5B@0?@X2pC zQ%imNc~W46TA0OjlnZvo_JdlxjxX1Pa=)NaMD)FXMJxf@hme-4l|sZ+zJ+Hk0MY#% zbOm@c($n2d`q`5G$M2}`uD1)1$=|o(*;8IBr^B^}uY++LKHroMlKC%E_YIu2M$+0< zH>EIxg^Ygkrv(~fdxuUhD zV$j2d)Q##-G=WGRw^9<*l?H~vB0ZpQcK9vTS4l=t{5gTUl3K_Yt%c>uG*jeyMplNE+#^Khm#@lh@TZ$wJ>Dl-2#pWSwO1(P;vy8%gV-!dVCoH@gNgW<(^&Ns?nc`+J^JKCn7t#C5uDL-M(N>K%xE7sSa}8nR~We6 zW;IQ5e#`C1^b~q^6g&c(JVi%oL3hBmGamjw1qVGm{)|Zusf3 zVxpjJg1=oqYJ}TradmmLFYst7*??BK!3i7CY!tBja0HKOfS8>q;W9C~qK} zwrl@h^+W6~hV6?xLmy;L=zWUXcs8e<+nWHK-&X@BRhoa6z=!FShiYn!M-`4Xx_`C= zWwkw0^}TH3;A6kZCFZvGijcFzMdClEQ@$f`K?gHuH@dhp&34Bbo&|n96XR^tD1LUN z_P%%TU4z4b4eYgvLL2GY?TC8I`8?*)XCIDoom4bKdbmzKUo6UZLFtqsVCHS1JN{i= zU87jh`AMfSQkA=t~19>BxwYs z4d7I?_=fh&OFZAvCJis%`G0z}BGPAh6zFHl3~)`K$TpU?MKkH>S4ZU~8(R&MHo@O) zL~r0}pCV~da(?)UE1@d~Z9}5%X8LxZKTSK?HI2f}Kw_YIn_vIu{3yM$t z;1}@k<)(VvSdPCkeud2>MBM?2`t%q2BLe1TdnceLXav6gWB9 zowsu#qskjiQd(=gE`1|oT|XYQ_&P4_j{lq?lh~ZUh2~iDO@)1|^77X;4?@w z&ATpF{pX_GSByS1M!+Hn_guO0i@Rv0+!9g9o&-3i(J1`++n5U9`M9c3wEV<7y06--%E0v`Jrn8b6>48n_1HV z7F%30hc`XWw-IJ^pI)Y#n8i2bJ8Er;-acv&Q`XYd%WctZo z;q)gX3MxKIkPeNVHWUkO0w)pb3XUj6JeV>Y-LebzUQR6YXJ0NDhfWkX_nUxj`~frul(6hc zFv~WI?6@Z%ctB>^J#V0uF7sgH+??GbARTxdE?z8$o2hYr6YuqnmkB}(a3<`1mc{XI zTiZEDa=Gj|r@G1-9)x%Aw$G1?KY)(Knl!AdZUksB*zE`3x?f{vSP2rs28@?2WTDnZ z21g7^XWLwG4xUd~Iw56_Zi#$meuuTsC#!TfkQ)FE3q%87Y4iL%pE5l$l`ud*c zN4RW*d!tlo-=s5;}t*5sp z5WrG1J(S`xEqYQ{HjLpwd)u^Wjn}z{dbWCr0!4(Gj6vbB>!yJ+$fXp6P;_`hzLO!R zan0wZ0SSnE=8u{&`FdKQ*x;m4COOBSqg$v)B`ny0jhMujj&1>o#ftav5u|B%f=;Vs zwN}ut{hC3|vk23Tjav(K$qU~|bzh|d(6!`^?-=<9X;z8xko;5-O8_!|dZWIX;WJRw zJCkx=4}3_p#P#X7TIE-3>`f=0k8)98$QHOezYD~+&TV!Ep2ig&T~>H^fg1IA>R%aA zgmq^qi~H>;r83=+F*c=y;SjW4#q14UI>oX|)5czwwUGlVDG7tjI47-g$^xw0&x1^| zVG3}$HQzv{cRp)R@uYvXTuKR(4J1{cka=UJ&~NqazJC)KRV4?nEEG>xJ)vm)hQWAM z7v7A#C9K2mxwiFoxZtIF-9SW0KA4+T$oiVf^<V=p4*#qMFrfg%$`suF2{Yd%v_>tYj-PXb3bul6L?&p5nAL=-lt9fC|k6~KbN-OX$ zpRe+F=0g0l_DKk|O|vP|g43FYzL4I`wJC4Q!w|U*@$pm+q0HV9D;~}E|8#oHm@P)g z6i{|GKv<-lcw6yf#>wiuuxObNi!xLn^l6`rTMHCJo8Hb8{$(^<8=!M<(UN zMGV4-vnPRnU?!Ttz2&0LE5+tEtWWp#qhr0i#|hIf0Mf{yGKH`&bf@Mz;Saarjfm-~ z`-CHGORqmHQYPJXXVj=b3DyKh`C*;}NJriRJwFVvm-Q3s=}(o&vOvlgZ*9m7(XFlS)~SWs;=fE5vOepO*pUuds~Rn}Dpz8^mu7BtO75MByS20uLi(F z3|6jCY2gnph&ywOi7AR4>||c@|4Fm)%-61C91wg01;Nt=*$mMQj?-rbGyL?$T;@MZ ze&2#!Ef`4-v6XqKFOqk)Wh0LkDu19)hDVQ?0tKmrB5zEpN)N0;%ZdiHI^q_xNt2q{ z?C-C#U|)Vi20_W`oiPD+4TNxCtbDr>{kt3YHD5UMH( z1R5EF-B905W*RJ)hVSGH*{H??NF`bKtMHg|^qLTR1{8o)E=%8-A-kkLj^tj}pQpT9 zj^-s+^10zsGw|F?5@X+ogfeY_WPRfDA})e}#)h1-DDKGgdBlv&I% zd?J_f#zA9nK2Wn8&2P0qyPa`&eN6}L1xh&(#_CCAyTIa(;A)DHon@age=*NwwbQoZ zH6_2W#I|8Izk;dWlI{EzYtx-@xn?}QZfV`;*>`$dW23JEIBFN=`g6HHx)uUmA6>rF z{=EOKr(gKqM=Z%mi?=7K&Lu&t&vOFXF&(0)r|i5h`zaDuWVov;#YR8XM_(vrdjAlZzysWGxLMz6pgMk}d9PfyS zt|)>M20OxX=gYn8#1gzOy2pa)Q$VGbmsD^XqCAE&^iDaWc`N>Cdyh4{J0 zDaYcPsIY%hsB^66W{6~EWleG~QCt4v86W$V_{b{|6fuXWtJj_;5nUHQ3XXMHf5`oF z>Y5rMmiAC#bKl!6?R3|2{)TK}2E=)Q1W|OJ^E9!EVWK1>$p7aKL;QN41_u}0(G7b?O^k{k`*1Q=r{jlUCAz5gw!nvTl7MD(i_t`Q_b z5!))}4VA6IoA2Lp65G2#ws`LTaiNXgN9h4kt(D|)q3sT!Pp0HkQI3bHeTXZ%gTwD> zUS8c7GYLtnF?JcKB;Uc#P=dRbwH23l|0E;wjaN!ZMu!yn>v96UW@iPZ-lir?gXe-U zmUWP1i8vn5_-nkbA-5*V4e$pRsmZcud3uiPHmAhKiK`|Tj_Wod9E6ltQIn;egRmwo z@=qTj)SMJI%fMav^Ra?XOKV&3kQPva4U(Iy4}vv>8r>^MxHS$Nv5~k)WNpZ>UQ9`y zL7iUn%%QkfKDTH2fPdd%x~$h@+fm=j%-h0M5s&=%e9z^*XJcw2Ern{_HrF(#k!S-y zWB;-_vN!?*6wMix4^a_+zGNXX9Is?_j^s6KxXI!t|6gzukT-qrNW%cy~+Xh|Zcz22owhSdE5>mkq3EU!f6?D<-Klmo{Rh=-d)`i?BT z_KBnY-s^d>`d6j_mPZ~2V)?bVq_Mfxmvr?j&&u_PEkI+9YvH3Sk&Fly#=ZdKUB|=z zdPX543+xpgNg`W2u#$voQ4yE5K0gyPF&XT#D4PREE)0>Wf1qW0+-N5)Y0_=FCoKiR zG77mgOoK*?=@4KFq?@9NPUj*Z2?xV|kc`)Rs&`I2ZYJ_QzzEIj9B<(#lA9y;*$PS@ z{yMbUwazvvh8-Pt=xXJBRgI~ZaTcrbEN>95DiI-%yE&8Iqs6I*?ad5`WE|(Ng?IDY zq?%~JkVY%cUst44vyiIh4N6IoJ@H1v6pz`@oAx0=+z`aZPCl#CAZ~pI7Oksoh(bHp zZYj*IZzwgJBT);PXFZjgUS}8iZ7(gbS*#~K3c%+e_sbfuV=dhqX z;+L9sDzQN5$<)dmVHMkmk$5bn^j%VKZ_)eod|9&5(pxMWba<4d%V*X_@$mBKxg9J1 z*u#2}&OzsJH-iIWK(>bY3b25s2Fey*kse-CP0a&U7v?=M?k{x9&1=#?V`tZFEQhMM z(Y{?S`ihMa7tswgoTj|@mo3-?Oog_HD#iXF5mRFd4cQbfhR{Byq!9r11PZgX7q;$2 zQcLoUjjh^Bk<>JzjrwgFj-NvlZ_f;RS5+MVjFUjax>UHAm298BVj>IyG~M zaLorqA8Y#tZcg4nq^T5BOd(Zo8i~dC)wj%dz}Z*ORz)2b#!4CZi%b{gKgmt&D@Mm$ zUnsPKceM+UX9raEfzjV7OitU1F>w>t( zB|XK|FAR4C5DgsYd*z_f;e{N3vpa+x(=qr;DC2If3eoek^ZN1mIZ9PucaR)F3|KM?x&@u8YLCgogP6x-u=QumHp^PjSF}QC<&|k4oC(b1^}exE%aqNR(c^+{g=T zoTA-Qtj_w@>;FW8y_aM|IY%Ra_;^1p2Po?+aTzx>+yicc_rLX-Re2Shf z7zK8^U5Uu3e<_hkZ{4Fq?L>1s`LueMEqWN-D|oM7sy{QXSunixC;9y?aBslANFy&W z+jM?Kp!oWR-?$C@M#h}aEGobWqpX@1`(tisBsP` zFiK-$3be{cEXDk|46H9V$f-fpcJ<7*GrC+HvkS@`=FV<47e#pcr2}{q`=s%D(9`x~ zgu{G}RF)`8eEjwp!$18E63!@v>C29-d{$6O^U1Eh5qmShbvk#U?5Hw^s$;ij3hX#A z>5A;)2(N?2k;rz}CEX10iOq1?tRx>|`&rmQX+}3=aCsGPkPu-HF#W`iCK)|461H|^ zcdR}GZB$<95wCNtZ`tSEG=25KqU|LWsiTptz_QI11Mj9KF|a~QNe}v9R#SN47v8}e z5f0Kb(Fn#nAG>OUUq2`%Iy+*f$4S{8ASru$aVU&(ZuK~qgw@J>%tDKp$&NT$Jrr-x z1|KJz&wMF0k3n;zvF6#_if{OOb`Ag-G@@OpKF3hV1{q0+iyLxTg3qvG4ED314!C&Z zu<)5ECw=mf?f;Lk8}GLq-Z!7%a~5S66N^s>P`ed-@JmUMQr^Ul_pP z?a{byp?jFzL9r1{)>0S(ELy~chj-_WeZ5ymU*9Hb0lEF~cv$Wp4~ur}%_I9+t$nwj zvCB$DUb-f=<;x0xEAe!&YN{-j0C@>=s;QFiDdy( zl%2qlGvd6k7HN1?8Q(zmVTDWgXsKkTDTcz@Ghrw<#y82?Kif{Lc>gB182)*qKmM~s zG3oxa!MX)HzjcA^w@Kyy)M7C%h4#w&h`#gVn*C1Zn*hgM8Zu5DRqaRyH7qjkmP8G= zj;;nD5mbF8P6k2JHeci`JO9fJW>Xgs_{mC^?HlBN-4V>MJR=ef>pl$^Bcwf`&m2ZX zw(}{{PsdK2tD6MB>?a1<C^*{{yeaqkZ)gZUfI}qceHWNCZV`VLz*Xpro<-*0FcWH< z+f;b?@RI=&1sEWj#FV6`r;=kPg8Uc3Uw$7OFbB!My@F9AsZ9gi#VSGem)(;_-S zx7Cl&Mx}q0$33>rKp9=yMnDsuDG2zR)GSrR^WU2%i7D z$iee}4qHR^Cf9e6?*vb?X^og~@ZNr=H02M~4 zC%rJ1ZGUGdGOM{U-0CRs&>K`P_{pjsTO^6djBLQ*l?)mz{d0oicU_~F>As&jdEK%G z;19H*j*mF?Dn2%#cW(m?I6S}W9y8vIyLUC2iSCcf$BxLPb)ZVfEvu!)%Ir+oBli-- zf1Rbc#RXhF=qCe1OP9p@t4m#d;WI(*>|8q!I4w{kbEW~ZdL@Q@o(N47Z?vyc)BY|u z&HcQ9iR$~NC1%j*3a2px5Pt784QNZYW5IH#T{IUC{sf38^0m7F8G+)=5R#a$~6+1%e+h~0Mu z8+1hRV1$dgbJdS9E!sTnzzx?Xh7+JcMZ^(Ix6HZ^OyhrMxV-}MMrG#9tD2Hk&n26mU}xj2BDEb`5yo(HBAZ^9FN<=E@m z3$$^t4X5Uh=?va%Z`~YbZC6j(N#ZeTV$E|Ie)fafSmQzDrzVRI5AHt?D&V-Sc`;>v z4z|P0+(f`z%&pIt|77#_{#vz}Uku#u!l7rzj+hl||3tu8i8N~*OFN-Ltdt-r&mIBc z3plLQ7kBf1%$O^d(@QzWf5%L-YU6vtx>ua4v2ltp_TW6EDkJmK(+%FOkc97Z4kj9 zS(F&cs$ViSm}^D>dz1W@r1vQqpW3=N z+x9^j<_}k1Q0Cv(`3Xm-x71eK`#)YKEOO9zie0-3(Z`uCE-q-YDyL zvo|v^0I?GbT&n$}`7~l{Sf#!yHA}Crp3Mp2>wP$o8MN?*LcgaGP1n=G7{Jsb-?UU7 zphwN#08;Iamw@Nq6n62Q!=aYk@sx8HZ}714U!rASD9+lOs&!TT;ErpD1guA2%{1M@ zKNa?$(h{BwG=x;8E%`Ny`UeB2XMVFyLXo;W`4SoS7P`G{KyXN^Qg zS9Iv#oNanO{qaRc$)jS)7f)fAXv8)Av9^r}`NK@}bh#MAtx*!Gg18PU4sLy3urm~P z8}NQbXFQTy_PFCY(r;4l3G`Pvl$~C>vSc;8haQ9U{Gf)6)LZGOPrw7cTL4oqCwzyI zmF(iR!43T!pf6M$aO3F?`{WAw$r+{0rSrzV&TKTo<|(TcXGjV10|?8;RDS(%zKK}^ zpAkA^Z( zBkn+4#X*yMD60`mf|=LPw$aX&TDR#8g)pb7ynLP7b?@56q!o7w5bnTY92Nk<1myx| zD7#@CXRM84$Sm7JYY5EU1-Flk2b+)fk#gimNt)Lu$3|P&W z!5NDfW*IM}J1ydbm*va1{`}_VH%k>&pyN{F7BqvA_)yT|tb+3AWlult;l^Ea7+n?pZM7(yLKmG+PNu zUamVloa2w68L_BJq+=bvv$?&4u;`=P>dxnbL(GhKBTexVpmdD^2a%7i46;AQDd&l>l)+O)wXOToa%OU4 zdN_<&vcQ1hwgKc{0oeXjINrc2z2en!{Obbv4KR;hBlN@5@44*RGb@Mb$LC$$%2iL^ z@Qlj==G8H=J2Stht3H@Xq&>3S(n;q`)Isw2jg@9jz?{MO-%O)?P-61V#p zpJ!p{wq_MHGm(X0+CPC-HM~0w0)idIF-Tp?!yGgJ7(fz?@*9?Jgsk%GIzg?a9{iMe zxi}8;ZVYJ#7RDe;b8_m%>s&nLHhfhv=%Z*{(rCGyt#q+yzMvAA&tXmCOR}*O!XgJh zdozy#%}@^T(__#vg^B8w=WMM`3G0!b-`d*S+PaqX+j|%TR=;%}`s}VIRIM3d|Fe0NZ42In%;@ndv$WA&64GVqnAzQzw-#8+ zR}8~o-d`*1^`RTxsaAw)v%IQ)zWI5|2v#Z_gL4|JH1cT#IXtX@y+yHwB1TvWMV$t| z;aqkv-3CYq^RB051FG2z$>P$9-0Z(cn|$}z~3XRuOHnj7v?lrlW6QY zO%6imk)fA<3fniSR3nN$1FRF)-suSo6igj|P$CBt<&WQMBse)q0#feTX&FI9Kn@Hq zWp>waJrhk+N*e0XfX8|H>V7id`xGWQuzhL2sidd|;tk&@%$yNj9}%5D*zQr@z7Y;c zELn0cXgYtC_DsyX`opNw7Nspe=>dp;5z`1iU7s7gPdIB`kuiWtt1G7ZhM_xT%Mt#p z-RxFhX;!g(Jh1;-_msfKTr5S&G-_#*%q{YlqM=QA$xPJu6G<7F-gA;it5@>17UW(! zOmP#uVW5s;><+XQy#!rz#3kA*HBj{)?oAlzqX zez3Km3`oSK)pS2C4YZ0nPW>>^FBlfv0-})1NW6Y1?Mq4EG)6c8I-hto6NZr^k5|xo*gP|2l*y+d^;f0^)URrhK62LZ$79UedjnG6}Hl1eK3_sDOF{*J2|nE{w2k znyvXk@K1~EK{oJygFY{h;XVDFzKm2>5S)jUQdwWw8Cer$SlJ`AC)HNrz#- zxV3Oh1Fa?^41dpN{nCD~V1UEwHK3!0(*gx|Mw99E`>pOJ@B*X#V+^7Z+w-1WS^?r8 zxiWi$-DY0`Ut0<%MqD(byk+;xZSjdz@LMKF<94xx=V{EayP+bfxBzsiih6e3v1b- zmRoUSewPiACo(gOw;DlC}x)A5%MM+YaJZr|XqH))1TooM$&y zU~_nf%GccY-G%E`t4oS?1-!5$RCPDRe}V}b$>_i~TOHdTp4275oj~w3`R)oUZUczY zEKA`kr$%}d+-Kw~0ia}MWx1{NUGNFnEOdJaqq&ZV%gFgld&xmBA8b_$rVD5)Rs!2XAn=TloKSmKdo& zy>&2K4w`&ed7QcKX~@2DIdOIgu5FjPV{ECS8GRFaS&pqkluKJ}R+7Q>c~!1m)*hSQ z>$=!1FI@wm*}=b*-3;-0_81Qr)G+7tDA(9D6C5a~6t{Yz+W|8f2ZwrE94>ieY=IPu zL={3~sn3wdLPQT)8`D-%<&#hC4~0i}0gu(W7aGe3B{skh}bzC91LJ2O0w@$s`W zxF1j(|3})S&AolH?Sy4kPhmqJgB784xg@|K8JCCFQ$Q=M16fbC+$-o@r3eeW+Hpu{ zm+||Cw*169;KAA(0f2|H`wQ~@66t;}4in?9SJngr1Km~fdrA!E9G3x`kl(86I&?*9 zbQpK{L}#B&#l<*KZkX`hE)}3R&c}Q9h%d+&XQiZdfW~v)n(FB(K<_ztl?G6%C9u;v z1L?-ZIOM<-zT+e>_E`85FO~9F>sDp@y=M0IYDaY5)L2$+kqItb66Y%o#r*oyVqs7T zJgycNV6nx+sasHie$?-_ayC`8alw7{o5^}jW>UlcA{`+26buRNVRPaI3DMTmlKUM- z+L)V`&dLG~F)t}4A2o%r4WMaAKF~abAVzmv9_{y$wA0!3M$;-z41Dj4G_dPP9es8b z>He&EX8Y@ppDt+KG1NZ2M3q5i?-AzNe46@Byq)K$f@kcjI(Zr{VVui|2E=@3VLOo5e-`jZG+(a&%R!_^{rA#;pXV;iA(z@3%;aQr- zr=icT>aPmq?Ik_Eu}Gdo;9Ox4Yx+y|+L#t_G8PjK1pLa%0xMmqp*Q;CZH3rZTA+aS z1CV*^?;Ny5Rz>;Y@Ru%z0?h@Vp!6w}ujCPDi_L#iyP-)}=lV`rA@6-}-7B+#$A`@2 z8ev$no`j6k0ao{EmwhXJg{~On(xTH$ldib?OC5NwYWFT!!|&%i>YmP@P^A*%T3NPN zz{TCM#u`nZ^4@yt%7YyptN^oQT|)xzghF`^FZvED*?-@F!(P9@trm@_xfOd8O4*RB zmdGai;aV(k0(YrP7lSWc=ZhK4?s<}#&Y_H9ptoA7;zei_lA=M4kbxd2eiK$e9$tdf z74-Jxb~b5T>`&$_w^kq8MnTHq{oJVcufM(v?L1}Qy2mJbzyu^Sae(Li6Mc>Vl+ zYBg0%4o7;mypI?_AAI&ty(I?7ST~OhIiGy~ygGRVu6Vh33N3OniQAg>3dKk-Vf|@+ zWfI62Zb^E5LGd?!GmMA@yaU~{Y}+l1v&TiX&+mdhoH*OZ!RX_0Fggd2`RA!~fGtv3 zfHGo%RNGH#9$m>y&?Uj)HlT5amFt@{0{+d6BtYHE&9>XL7wXCy~U!&j$jA314tR6lFJa5x8s zT^-h?e`I=-8JXsjR((0%3raaVlOi^9jo;Sd$Mn;_Dc|kl!E9}c!p5UoMk$ZSgvJri zr)nN-7}mMjCf@tmtAJl)v@e!#z)*LDp*E{krCz45OOXS9DPIJ8J{^lUW-Wtip^-cB z#Ht_D88V;^p$$3ZR=TGQfeY-;h4`_~nMw=U`PL}>KA20EG>FPcWYmtkOGh*Vy~z^d zV^eR3{BBz6_QR$(Uk=~z5Wa)o&VjIUa&pe4zWH7tX9U~oE}XBP5$VZz6eQ@|!A5Py zB_By2512AKU%*zUZc>AlOB!amq`K>reo&08eCUD3oF2{7WY0%9N^yyqv;nQa0%GPm zB6EG}KPazH)v{je&;b0IoJKh*2Nj$0_Ky#>mr>RyixW$EG>%&m^21h3)20&X`H%a{ z*~n+*Q4vLi9LE)jwOyT1XyWWuKoHCRPYYtz0R|6xgA(Xq_wnyK*j@jtgWW@Q<`X$M zfp#d%BSRvgiJxu&&c(!kSX<03 z(3B8=6Y@{0$^SvQ^8J77Z~Fge8Di?ME}lRWG4cz4)4OmAplS(CBFW4$9xLXI(E4V6qS^m$>a$mGQYJQ9@G3ee>r!?0ryBkW+P&HyUf( zmC}T*cyiZ2<^`L~tAD%C;`ePU@5mPK{UV#Y2qT@m`q%v{|8`MCDAnF!wHc&;XLlNx z3q$UVF*Q-n#BzfxnlA&Q=k~RuOFJM@u~dDFA!)z(s90BG@1E^ z^qD!&^2_KdIR(bLbjWR!4NnUSq zoY9XsaU@7@Ff-kR1&_@~5&O8N%w0qP$;oeaWzY1-{pLN7@}%bm#2+%G54q8iL-Tr5;BCoD z?#2gGvbj)~iLjF&e}naIZXgueluo^xck)mARrCeS4{0%5gTN8mcW=UI7TbcpCEq;a zsbM_%jb44p0ERH=LoMXwuN-B4G4a$_LM4n5#NG~FHtQ_^Z#Rnnw-xwB`O1~Bfb&kx zH2-t4{p+buIZ0fxsG;(uBbCDtfc&;e~;{Bn&^oZNZ3+WU|A@zL;iHl$`_-D%u z<7MyBLVG{-n=ex;C$Ia$D<3EIAq;q=2lfk68r!;8`t5rwtY*(@k6X>% zgxKhQAk(U;$6ulbR1XYAbNM9hS)X2^0DKfAjJ(39o?WPrf&( zP7G3i_d$nK&3hA0GJYf{?*Zat*GUq3i27G*ka(ZixJ^-YYj>Z%#7bgpkhJ{4jX9!X59Nl&Vg4BCyo;KpOb z0Zmr9=8w*L+S~~A9i22^8!ll_v#sdz|+;wWt~$(699e_h_e6y literal 0 HcmV?d00001 diff --git a/examples/grafana/screenshots/maintenance-metrics.png b/examples/grafana/screenshots/maintenance-metrics.png new file mode 100644 index 0000000000000000000000000000000000000000..41410652f766a5e6d9e2b23828ab854927ab24ea GIT binary patch literal 390013 zcmeFZbyQT{`!`I7q#~%~&>;f~(hVXF3QC8xAl==ef^|&<#T~^t=bX zjrSMdb+6~2_xx}uH3wRswjnbgY*Ux5)z(_^kZctB%D?xB=kfq z4B*Z?dJ$ug(V&y!-h$t8kGHof@Af`kSLf-0zD1igsT1jh?qNkC=wAbihZLf+wx<{ zKvZ>%sit^m9^_9JMSM&ZSCa_6khrIGcaVG@izPF};Mh?MVMN3zZjeiLI0u9uYSVn-bk=3p~L_GIA8wi$nH#!n34t>WBMTB6tXe@$FMq z-w6VPwCU+C@i(jD7O0kU$qre<0^67=p-pDlc;@@3_x#m~<@BgA-z*Q4UmYSiDG+r_ zD@+(G#X&EGR7h@Rgx*`(mN{N}K(88n;!g*rxn*F9^i-R5t4z(nB%6j#*EoC%Yf1Y9 zXLUak1?|Aw<)ed#;$EV6+k*&mpxm4vQvG1GmX+^tkE7(zvwwU0TA(~5it4T^OsamGbTp_^5f?8w0n0WC&p=g$JL)AT?I&^E%`5AoV!VasP^@Oi)9rBJ97*1 zK@Ckxxlwqj;>kFzqWJY{hs-h7DGzn@2SMcOv6qNcs+Uci=n^AZyjEwXv$YL^*<_h2Z{o zYbo1LA7k`yi@fU_j3jj;E%ayU)<%4wiY+qpKV88;#!X2(mpBeA#v?O-3vR`Kxmp}O zfu4_@hsnCdJLXgIQYzNPrbB2QVvAb#_3$(r&oKy=uya4JH*-4oTkgnu$QO2{M_{GX z2X>5dIX&(75?opFFAUzO3%+f(MeCXDnj}VAhD#l}xf9L6b;Le^XKPNIQ0F}BtE%9g z&TNOgNaxe@_vd$=`-AXAz~)HFBCJn4_g|0~))USp`QxVvd%nc&^~DSAxL1wLK8J}S z#`Ffw7KAMy05xs(WJ6VHk$*sN+xJ#6c|P*;OQMMzUz)9oslNL&*xX7+k8dWnL7fh) z%R#M2nwg_G2?b$cr&HrTrg|vGdegoAo<6OxH0V}XJ4i;FCCXQt5S@c4Av^|jKhn+! zNBRAh1U@ZeB#tzh9j5CW2b!WtC8nS+8T_Abng@nGo@BzrlUN+Yeucr04i>%hQErH% z{MJn1SUTg7+!|gvg{SZFOC*)H4pTZ^q$x-OPyXdgsauDF{r{& zXF@9Z4W|tW49VFBi!vN%m-)oP16wytZQk+3?1J=)aOYdo%~xm=enKtL&9tGkMuge7 z-eF}0YzIgLaPDesfa+;)y_NsOHAEpCBGb;gfV*&O0b_x30o#CV?_o*sW198oEg5L0 z^sTfQDHfUn+HD#NMjd*`2;qp)P6i`WJ3_t$R_Z%Za$N#l9i3`j1D$EiiMQq6G15oT zWXk8TW>{oAQf5=(XG+V}eE#|i4_8>CWQr6xV@!GA>CUs+&$+n?xvrlj^WKiGe2LB7 zej!kF{B%2`wz4*bUrk6-C%y5xXYd*Jk-&!at4E2%ec^HPJsIGv7bA*8l!L;r9y}uF zav>Fe&qqp3>PV`_r8~^*Ajcmp9H}eHo<;RhFWcEn`BA~6nnaJj$wcDBU@mL3<5v&M zpO{ISFPnphrg9y{Hfy+LAhH5awLUk9y5}7;Z+GB^>i5rapoN}>T846l;!}JItH4(R z81u1dm13S^Rbgdeh?O7g(aepRjM{DaP_wT~5A8qnd^k_wNk|L2hqs7Jgr_B#?=&&0 zzjG9{dAk1g@ZHhz+RWg2=D;w)_kl6JbiE?)StOof&Ntj2%!`NSLdC{yQs#Br(Az?S zcc`wCsLfxQ-ZkWHv&qvB(GQO-c~%_x{Z+~ka|WAXc1fCT+@ATO;kOh9Tp}u>k}mV; zMF#I;2Muyn{Jc=d#8FH0SEL4C9?Td_nV(JePO(Ds2Q9)UJ?+2nK?G+!DqPE4s{-GC zRS!uZ;Ua0$P-dNsONe_79?)p4Dy@_)*L?ND`e<5nAkkFkrH-NJK#E}p6z?y z_oO;aILE%MJQrG=uq)-Q<7K~0xiG#8zL>Qaswt~xL@D-}^I7m26Wwk;Y2I(f6f+UK zC&nc<(^A;N>8BJxDBdcr8Zf{Tz-KS>O305}uKtk^PwBEyEKjNwvo!@9z{_pG;(q8mUjPK(sg$uNriSvo;3D2Nd!YHixn(iU&-)AWFjExqjOJQ}%?OQ0q|39- zg`tpf@RcenYoq$Yb7jqaBWrs*v7)B^|-vIefsw-hY`&U&976(rMsm;jwG9LMd!+?8eSQ#O$GyQZAT&F zO|zhB8^?^NhK9l3uAQDoG`bnK0AJ%DGjd4F=C&{n>g=i{O-vE58k#G zZWey(KDa-3CJ`#})lqupuoiaKdv+X@8C2Pp9-KthKuSPiD_EM*Ri3->88&3x@wtP9 zd{ao*rhYZx@bP$8XmofIBiF1D7wr4d$WV^fKtKwgH}~3Bu6?q7$AEj9b?uz%{I2Bj z*E2b*_zg{ME&R)kv(0qXs9wM!mFL&QDX41&X|S5ZebcHE=BWyPelicF&h5ANQRhv+(k|3M$XLyT!}TwleC% z(ZlrwBSlV!OXirj&DJ@iO-Bb((s&#(9Cc3(FQ_QkGuefmi7#`HR?p(O*$J+e;c9a5 zYFL(7E6d&br*B8l>p` zY-{7pX5aR9?7s0uqaHFYQ#kiJQauzI35<<|A;_cLk=GyWiT*6b?40mKN!>AM>%JAz z_g3$-HMSFHdmNtz2vV`%+2L)ZPp^>LU6C9P*IM{(;x+}GS#WNq;&C4|WiMy@daFD) z9j3aVoI=U^f^&E33dhpSUj-%A_w%)PVXQ7=qM(4p0$gJup&^qZp#xXQz)KjJ?DzE( zWF{okANNs^kb=#S(Ee$o2z*}uMF6ksK0iNEqk@nyfp52fm&-?#U#)RkKcfD+Mo$EO zLlRSwkdXmCRSfNnjjiloSvxp9g+PHD*f!D{_DD!1^w%$B8D+X%;Q13~&($5&73BF1 ztu5K~jjRof*<37buAhS>=)wg(NtQ{SMXlSkn z`u+EFp2jX_e@3#h|7Tgi0@<&BVSmW>fc^Kgfv$qrcln>1xfok$JT|igXaqBNm7W8i0GnO^q4WW7Og}$?Yt-%H6Z5qq zBqUKJna5(!U68kC(0y(%BRclj!5YLswb*>Varf_!OIX$mSamm&#^G;1k7!Zu6_x7c+6%R&2 z!2VN`X2zn>ieBCIBldmvBs*I>Y~qNLb# z$q~lXSZPmI@-corKomrDgANU_W7dR|AagD0WfHu^RD%IU52`;)6_osFLED#i!a z_46jo%9)?*>t1@5_B36cC1pNs5ifnU=V#QDorA($GNV;>Wh}A7aAwuC!o90VC@e%_ zQi=Y1V%N+2vBv-Q=8MY|LFfXvs;3|m=|Y9#Qe_A~Jm={MeQS zLO~u7ZjG%AtmbBsX{xV^qGo`Sl2odt*#Ewi-)jV2;I^>MjqAUwS~km%@;|YhQxK;DkmOHJ*ukX;(g%jc23f z`7qWSLJVd@T%8vQY82JhaiY!ZyfbpgYGsr&!m)suXa`M%%x;%a@{3E=P>8w2 z`%(Ty!~cc$z=U_f5rj`bSUY5q_ip~#8)-$*DX2bZzU+gb$6nNx-|kG^QDhHWgSbg= zGRqFzhaIa3jHrs(eL>g#?mXI{PBlGm{hUCI`JNcI``(t9a$!^=EB$0s(f8HZ_^O_p zP1ac#ZB5tEI+QMTMyiyW^wA(L4#H1+S%_J*X{KaPU^8`o-6ZU)+F?mAz1B`Xip$`} zfbCPapk;E$JG1Ov6<9@^{0F^dMh9_yGBB+c6m)v`F!EBvj#46ej0qid}v|l?Ot_=--V#G^xw=z3D7!X-%e@n!ua1hoR&z^RhT~)l&62gLsPr0jD zihP*NXFr8Wb#=ZY!mL^T;Wk+L`B?rX^Jnc!i^;8$>2lW$R}5N3At(o#?o-^6z#Y9+eUSEa7e zXn;~Z8#cbG=fOh6_};OHPwb>*@ri63hJwYM9;5lB*8n``s0pzW=jgYqlC2nXc0^iT z=zhEvH)JT)`Hr|}x^ya@fsmMOwZ|&Za-|Os@nH{k!-F zZEmG^8L7PM+Z*{nN~wa%c}~Py{R3&+T2(=4^L=d;Qx|nI9hjQF1gu;6jYFvCTG{#JUMZj5!)`4dyQ@=b{Q zN4$E-X&?9P2ap$^s)ZkY95^t+zR_Cs3aV*UccfV7sUSj@s%v6F+BdmkH>53KDp~!AEJwx$5J_|yfDy_PJ%XiT!bxu=p6YPzj{lM@dv&lxQ zOVVR83@{m1Yg%ptH&c;rRzKgk)s8pf@;(KBMPOafw0D?Enr zIqkE`Xuth>R=ivp!$5cCHe~7as$qH=mx^0fHilP#sU3#)&HG}{ovhyJ#_N#%26w#T zmR%AsSx8WEMcwiW#6s?Xf4caYPR@x;TI_wj`zibmocfoB! z)IL3y84*7}sH8tXwH_#0Oup2taFcu*vR|a++E&>QvkaWYZr$Mm?`*oE6kLUzgM8rB z{tWLEdvu&<7`+h<3YryWCgWM&nvXcet{|{--_y!%t%<#GVP#25_wp~S9y=6e4ZhpM zs{5w$NtT)^1Lj&DBDM}<1N_bUApyU`qk=1enFCL~ zi2Y@DZhzZyteua$Z+{y>e<9|7ySJ{}>?$E7cH-duc|!E}4*Q1Qk2n)^=~($jW)+vS zA!4XS?EH;_w|Ev~jO*@>18KlGN%!XW0$6Bi+G2-dtjFno(n}{5qCqI-qrQ7g&p#JC ztbf<(juV>2C<6e|f|1X6Z_t%Ya7G>0v_qXBXq;S+XrVEEcx=CLVrI9JVA@6G_@z31 z?<0yyGj?dx=@@PTkGIp{5@1b8rwY3IYX%Uoyu^6KZ*LRb9>`^Ns^h^m1Y7b94|p=6;EIayVc$Q zl%G0;?;w%SGwdN^pd6$(5=X&pl$mU)=`&t@K33p8X{$K7Uesi~iY+8;MLnXD?Vzzf z?i^vZE>z{iNWJ8@jC69nBanpUOLeOBCSz zZ0+AzJ4n6`^m74FCFp0?sH~Ui2DXe!FLD1iSCeZ+Z;72$vod()kMs@L9yBdv#~#J3 z-5_CNA`sjyo_6gJvK!MleXNCLJ)WLr0jV5i)Aipajj;Zn={2~a>$OMBY>6_KqVRel ziQBq-q@3$WlWS9Yl4CVt(3O&tEks~o;VX_&_a@PHG11+k7w5Llwa4`jY~Cb0L(!eI zwWr^HG}9BPNbe_B?XXmqDDIE!?6rvF+&;T!XghvTwOFHMsSu|~oxyn*L z1Xr-DpH#}HJ^V`I?6<4LlTRxZhNfBQ9LMQ&Xmsu)?r*v=p}O5rP$OWRti)Ijof3>F z7P|5csDst3!I}`%L?-O>R%I&CJ$t`h5jHIr_QvnC2^J!N5E)Y5=rqs@CrEsW;h^9A@1vbr;h9yW=|U z{^i!=lfBWBVU$8FkV@-V%j~0lzrY(^lFRp{>BLUr^+?&W2Eeq}TK)1I9yrsa-=>Fe zPCTR4iMXP>Pi#|8}&14?bZgxh=kJ&Q5T@ww%tt}4lyyT zu9f9H1nd{5<5X68C;U28F9$J69WIFD^^Dt&awr4R4E=T|6(3?*Q70DlqYkCFWI3Pg;C)tZCC67u+uK?njx)QX*u^VvzyLQ$8C&3x+-K2XMV zwF2d~^0lkXeC---4BQRF94XQO@5b2><*U|^r8a^)JpnR?Gsab~#sD?E-T%gKSAmo( zz7AHaWYX`Z19f#ytqYZQPvkOVcAmDa>P_X8Tj}-Wd{qg6%m>wtgIaLw9k%D{Pa3$N z0p>SD0fak#rp|eCpR_ZT%dpVw9wHzBC-pV8qlHD~QKi;S`TQxjiP!c8+B_z~J-S); z%d!3eOs^zG<6Kr}uZZy?QWwsn#KJHbTNP#$u z;h_Vp#!N&rwA6g`M%7lQT8GTgr>}HTpo=&2hMDRmw-^hQv`*6mZL>8@$i-aL;{K)m z#B{BLyCV$*{%irqN6`(8dE$2L03}b)Ppq2t&0PGOfJ@LhcQZ=F8~NIn^*9+ichbxh zJGrBGm%+FvnQ@UZG%8$J*I;#Y?hr5x=$v7KH__*dYD-HU)~6-#sf6hOOQND6ebaA$ zMM&jIBfqK>Y>GLBzpA7MAI><);(3S6%6j_HjL!fCF(q{T>curSY&&cI@G!))WOA71vvDGU#lA@EYISmCqqh>J@&x7lcR z0$9Ls%7QG^dN|*A3sN;6cHg-rkmDmlE?Y$IvDitaC4U#*_$*yg-&sA>^BgxOg#F8_ zI0-^%>ptdh&-4$E{`V`67$mV1T*vAO^}p08J)lNUNJrBmHbzs6S+AvAq|zb|1^tHX zWa%md=_p+SGt2BCh+DoekOr?+tw>eQ{CN@h+EdNHGRwNmcjz*{w#z0Q!QAQR%nTky zItISBDPs*bQYGm^kzSCy+M6I(X>}>*rG*Gd*7{QJIA{hq)hSa)@*f%#FAfHt zX_#qh40t}VsySJBH@3NH-54=ngg>NlYo@5yF*QnxGM5I2q!Bz~DnImz!>rYTYWOSpv1iGruL?|80S+6(s+AuMGg4sZpCcFP^%^g(Lkp*qTk$H#?Qap82lm_3 zrKD5r04D0JY>eJt&i6RY^qU;Z*C?AbmK>(37jAwqRUYRZG^vC)f~dSwJCBfWM5m@an(y^%{5+AanS)r}*FOoysLBzXCqlN$|Ykq}hGV>J!(JadfCkP7F|h{kJmUi&V4bz&F- zo-L73tI6JXgw=P1Q$<%W%oVgaSrd9~t#xsK)CJ-sRvOkyfY~CGCCA1kc0dkWBfcNs z6_L*V;lY^*tlCBh@CqYV&l^F;J*lSRqZ*cxK!ix5GGX`Vg_;i0?SS#PVfP~sUAyHQ z0Y??PUqM0Ulr8uv4_iBt- z=;O7n_NQo>G(6Nv%~zQX#X)Q2x-DXL!E}0DXT1#j^2q6#5VbIBWb%GYFSPlu$Sj|M}I$X_jgqf2rG%`H(8z z`d|ApI99ocf@h_L3rUmgNQgX3JN=FrLA;A+a_xQzcs7ymh}j~yy{+xrwsLBpS~k=h ze3dgoGqYIg7h^3F}UEDlg}*l*)w^t?_r z$|Wx{YfFv$%c6O#rXrq~e9kro0EOj6tvt8&6qCDrwNNhG>#=<07V@&AZ|&#J;%kDY zDX~i3dM=R`&{7$vtrGzUyY{v-3T8E!IK<#`Z?S_ml|9$594jpbf8?*#{8v@@2U0*4 zXdD)VSKasThg7nzJCv>VMopHP^7+enDc%*)*v%lh*+33UESX3i1TiUh6AgnYi0z9&y{~!v#ghEsv$7lj{k>U)c9dMMv zYNg^WR{HY7n|IRR@qF7^rmE0$+76Qd7XM|G(k}S&*kkv~G>TqMRp1;`j-)Gs+W&&B zJB8h}pM1DX%6o`}zRG%*rA?1wK{4YGl7eWl2m=PNq~2Nzav}Te=B#u~{~G;2PH;{M zB!M|!Cp<3jswi!9O4sXTzV4%KIw}#PeEe`$Pi+PlmR{C|Kdg8*_3VgD_Q+hR69|uG zTGp8w??7j~?kxAmi+Fnt#jR8bc$`EC9-nx^9=ZM56b?TC#6xXjCcdc0?Bru$_8{=o+kQq5e=V`u?E2|b zYZ|tR*n|N}oaU+l|7sXY-w!uhJHCFEIU*(Ky1G0Us{1f_t{tyyH?9L`2z}rwgMJ;{ z70!J+e~Z}>R+BwVeR>q{$l(mtvb#7b`Dnb>dtcz05WI75>v z5kOcYBKCfK-C|1W>%tmEtC~|MX5CK#TN5P`V}+M8!WTA?K}=kgmcM0fmr@*&Zn>?>X?wG0&?(b-QJL38(ET<}%ZPsIq>3IauwY zZq0G+=5n|QB0@>zQXNnb+i?Wr8Jqc<6+y|~UVL4VLc#f(4dUMod*b@$-_(@{4D@Fj{ zaT7_{u)GiYs$_zX2f7kWQ;DCeVlFHwcAj8o03L#X?O8dWa=i<-z!%S862Kgkh0Nxv z@DP!B4<0KeaOhIH=YP4)sy-o*hbMBhHmT9bneLKvZqpU7xFeIM=W&{QxH;ZNk;bT` zoAAiC7H_PMcG3OXpJ)pv;GRE(`!k%*u$fZhkv)nf?y0g;`qJP|~Uiph@(6t~az4@cf_bBHru z8K7)$Pw^0tohg|&65o%Lnf01cbR(p8IK`#?BNSMz=k|t$p>zPS(^7ydiT{Nmpy9F8%cG0CQ5)R$P1Wt(9tBsY$U=Jy_O z4cHu0^H(ZEivm+X^{_l{qI>2WtTvPTS;IB$T9W z+n>UXgfFkY=jqWp*#4mpGYFsyI-$m6{yP9T_Y)8%{lAi8zrfH$(T=bEq&WO3jJ}oZ9v!f*N0Hd5oi^i@8{T^c;t!aJcM(fMB!Mwexuh z<7(FV_T@!SmS=DrjKBVXevGD`1X}Sr7?22xfIaVa>I*%}US2z&+?5OAFzpRT4QenjvM=Nj_lK22AC zIThKaTruUpzte=6(zhnmm3myObGpAAT?O?}D>cR`M{Dx<8oodEx~TKp`=*e}%p(1x zWIogaSU6`v<3OrVPoWZmSIot~{42 z9bGzFDMcIqgH118+Qo10W58Nh5s6%>fXSU^g9pT6K|nm-Vihk%i3;@Lk!$kIxB#qL zZK`5>3D(j6)^O=PPfp!t=5Ng;eGgDa%!{4M!({+yA9^@UaQ_?8H&6rh`R`J(ohs|5 zam!>UAObfxMqe`>^&$A3Ea>m6vf-$;haFTN7Koi7JnJ8)T2A&WLh4;J1V=s>3Rov_ z7#p{qh<2W?4P_`7oT2VRTa~sB2SFFHYTDo`s9Ui1OTV24H>bFU^W)l{n!O+`i{v%d z10p{*IRcO`v?c?o%!}W`?Q)2jm7kmRCGX(#m}{$XTTZ6N-b%WZuYyJ!@XsbmFqRXv6X#AZt5X5ASkV3HLl#Np3`E(dR_D zOudO6y(aNLLGpjU_lblTEao!PeGOY3iUoe$6J$4fo=YR(3@g4IfV;~SHeEe|G@V=qW;^atzr~oB7|&<};3F0Qp=dzibTz;QUxmEd zz3|1fuDU$yAmdfamprL>ttPk0-8b!8E~WKs956g^@Rk|x1Qe;4MB#cHho-=D(o|(3 zPGE~BxKf5`qFjNd($q8i)dpt6Z=uDP&tGUc(}^kdmd<9Yv8+y7Bwl{ICLCnbbAdYa9 z#bhZY;#pr}c;`HAa1?Iq9j2$!q};Dq)pRv9ZrowB@Ehh+;=f97S(oFG7zLQ$sm=j;nY~_mW7Pc#8>^n~jV1P%o)T(F z>V&#lLJjanEs2nx;`Ab!EZKO-UWODwyz7A}#EUV%5H(!bN$F|GS*0kAHvpr`bgvE< zU%=ueg7Gol8~$cGUM^oczg-5VeCiWX;E4h3w&r;TR%nXR&BvCIY1KJPx!o0J0>A)X ze_5>gZ-9m)QZx~Bew6zWu62iwSC(-&9a<538~VIl3={K+Ex=|}?rNT|!y8W*nQlBg3)6(ODLkD||`SY*UPK;5l-?zdsO z$1z1cpAgD5s|B3WhF6-kP0X)cW$-K}iggnod3KZR0h)29@OV2d8WZNtyY;313G>Z> z@iC}~3&OsxupQ-ap1WN@uX=s{8`o>7=vWbgYntxcZ)aQ>{T5CuCdn#Z>z(`01ce2+ z>|gW-HJcF*R*_vaSh`CSLh%y|(j`wuY^2rc{|Kan70ufA_}Bh}_W%Osa^=w>!E3~v@*}Xo6FVsvVadPw3q}8T7Zb_1m)2&= zZ&!9JU|ua>ona(fAqt2}NV2A>6JEjUVDW4^ZWGJo!XDB<@yq712;eAHijkvNdgy( zxd@0^+!a=$)C^L(L~SEgd?wn|Aw@t{N$sN00y(L}Ykf_r+cei-9YDPPibC$+m|g&Q zWd<~x;kBl9x3;W_n_xicvF-VgCtvJ#ywUYuW?RrNa`t`uf*zSSlr)s zhi`KgEQd$KXZ2p?oa42^*M**1)iM`T07XU#hue@D9X-H^H$FS$Q?7n`Q-CeYZx;xK z#!FEZn8&+boY>}S71eG*8+vrC2U0JW$F!`f+C(mPQv}B=8n)s6Rj=WSxoSn(6^;M&GLNTNqZuQ-lpTO;}3<+41N#wG8IPCSQg@Ef;iFu_S+rO<7Q^bY0EFDz6ri?Wsn;Ka*JwY#GRS?g zDZ{K?{TYZ>11wtvZd6L*ma5ciPtK+!#ep z5)Zu5^PSncmOyfArcUY#-X&E!T<<#6mFZn=H-?1z!H97gmKuBXY!^kXg=jDn!(q8!ZfVRdLO(Bfsj#~I)!R?;q7rX zMqXqW01O(deIN2~v4VE*eV`N!qy7afztz(J2n@|dqVgE?IJcST%4Xfhxvnjnax=&@ zg(53f5&DzRyNq}Ao)3XHw}%L$iJ;PXpo@>b{#ka*n+-|&sAmK4o%OKI(Y&T7)3N=T0YB^xiH0V!tA3kOu%FcG0#wGA6>!Z%Y0o0rY;`R zO0@0P>SzjoE(`Sg_Q0n(7yCmMu}j^_hdPEiqSjr__U!3xX*juzRLye508s$Jq5BgT zHF1$&PyNqE%bWzL4PzeAqHOEXf})w_ir3yzfRTc+eYN&1dU|XS;{1v_PZeDgdt96} zTep5qMeAy=U^oUAF`W&%&zPV9jZPDas^1``MUW6-Ko8 z&KNAJ`9@W_E_=87na*<|Kus6^Y#rR*4k$+ZXomY+Ei*Gl5IR+v8Q|4ywkpi=4rQ|Ds?eOYo=6rpK#wWx;MQ$pgPf$L^rjN>pxz5HO<3>xbRz8gp!-~pxk%p~ zo(Hd6?*IwasA2iPGJ^ksZ>do3abxWm>SZKKqOQg3xCP$6ud3~JWn2I#@`qneRWf!Z zm)R@Rqw&bRq76I3(#qcf>4mPqWwsa^$#-2Pi-^wd*!xRq-eIZF*|MH2bvp)W+xIzg zdoW}^=e7L~fzt305f0S>6^oEhi_srlA!0_wH#2{7WGQu>k(-Guvl(|FQu@ zP^k4?`f4`~1PfXr&2?e+nxR#fu(XqVC?)Xbdj9wLzRw9C=oJTa)~?%Q-a6Z6igt1q zKEffLCiQ4_8Y1=ZL-+ip5B?|A{(UdP4oEzz0QTf>YY!;bd?O0jbTM{Xfjlw`yaF%X z7sxz$WR{w}eU~Sf>L`r;I`&ZC=D!`n%8U{9NZRrJI@#{-!5|#rML^^?{;3=zP5FxT zw{!kK!qzY5h#~=^DCGoy6&?{vQzKj7)s1fNFUGEf;MNn_@H~Ty$_f`lPrOXcLHIis z${zfa_m+r!RY;^yviTHR>SjwtM_^8Pl}GTCU=9>;%jPRaG^!IX$U`R+XKW|ktKaPY zp9#WD4TN^+Z);%ug)m(=cw69ee$cpi)PiOKF(V>MZlaX*h#D0Cc}9#o##p|u(58E9 zA&dQ%9TN{c=0Z#jc6{^eDICOit!yDgo&9O``E$y+f06b-q1MoKRtp-+_b0SXxXVmP z8Z_=x?3^rf8}rG!`T+lee>*g8VKK)|&ei1O9pBIBt=~pxROg5q&C-!wbe@UN&8YOV zJ=jvtfO4DsN7DZpMI0TgHTd0w(BC}aBl_iri~mk74c?8UU7NWk(I!q!5s8jXUVgOU z=gmM3uigutq9jLnSU7^R!&*)Jfv^HkOzgi}r6|KKV8`F$bJxB5JzRt?nt0o8>b9N# z4u8J_Ipgh*zT0RxXUb}z-thsMHb{{^&8(#OZmzq!I{#mL^S|#V8c?5bpzzn~=gP6@ zNP^(y`4m)-@sp&4!H`~A&ODzaSAq?`(~V(3w?pJU9mVa;i(Uj2ZP|+7RSG#!TMuhp8R}-7JqXyO^ZSsBUD*X6)Tz2H(;)= z%uCb9neOCYfwph?bvCLJnveGfqO#M1n_{J zTUBbs&*V)6Gx~zjY~y9A^^UHdyOIzr0eKi00hU#Yl_C-^E;nx(1>dtH}8sq{Ao# zNV@o${QtpmW~K;EQ^l?%o+2OuLAUWh@W|x`B^Jrgb3H)Sze;Y0XMUK*KWGO^!Lkuf zRm4Y2`r%_nn4v2!dnCIFj(Ay4*j{6+uLF&!(4Pj^=LUc$(CX-X7dsg?mhI6UK><^r zNWW$NX}pPcSWh0Vs^=W3>(8cp;I)=5E!im{_b_gMspW(afip<)3NMBa0llWG0Cd3M(^r|J1G#{b&~#qH}(p`~q(*@UNF6 znoYc`5^k!=>O}ry7~i{ua~^^i_%wAtObtRJ6tDh{+z!Kq!7`#B<30q53Q?kBM(X^) z)6Q}fAiykLg4HU;BKgmvh7)|!sIat)l}s@s`+k(e{%j>5@-^%yuMwi6B!_o2iy7Vd zF;xTc=7DPh%9Fb}pTPQ!)DUWr+N0mbv{#If@@dv%yeUOe^aLFG#A^#Y1 zf4Nd?%+f?N&ctRK)L!B@6TyG1T~-jMGXeDi{x;X@Q(I$Q^ucrB&sHM_ zC5u0&0c`8)U)1e`P($eZWZ+SJ2W5fzHTtJBrb&x=lNq;oql^3>d0F*e|^8Cr=T zd|A8f;nldvZ|~EDa(nj-JB<3KM)zl&XFFHGNs5X6SU*Ho7 zeMh}0!Jmri&C_dKo&UuAt3Yi1DWJ(*)(WTH2T(t)u1SWF+!2b0R=DSQoNteFSYK(` zEwsNKT*&2~u5@H%*3SIQs&(9;0pEJAr6dckmRn7|KG;EDQ*8;kiW+~?V9l4AT1}u#$LfVmaC)3^8EAd@4#Bff6G++ZE*Bf{_wm77##_lZU zdXL{I)2J)zMYcj3opoQT89~s2wFVvBYUFI~MZUU>xoQzDQ1ckW+208kaJm9FA+C~u z(+BjDo9X4yB6Z!_7iy|;EHBKbbUgO5R=0Oj)S-IEfWu-SH|H?An3yu`#!Kr29PF%c zEVnv8*(;(?n*ZW)V2hP&Pez=yB$Ls)R`px1TyJf(~v|!Rsv>jn&%s8{|0+V*>=>2?kF`@T_zJ?u- zi41+0N$&|Yamm+!PnwX0C1D-{4}Z|5k$WDsq)+n$q64n=J=PtChu16#NSbsoiq}9# ziVh#*!fwB{IXydYLa_hyM{T}w$SrR#_xfO}G zwP1eE!jF2oqn7p$`HzVlDXWZE`ZSE7^@$_LJl>VQ7keEfLpBS==^0zqeD(y0V+4NX?O=qj^2Cvfh9!e}Y` znCtW3{(6+mt8pw8PTZcno7>JS-I-m6ZHlHw#iB$byUD^1Rjq=r;PX;Dm`d{V=_`Gj zJ^loO=J8Y2W=tJS1zMX?Z#eQ0a;hJ0-1_+2aS5^tUdG2>F)lri1Ywd zCrTS~Vmp%gcrU85cPK+D^GShJ6l>Zu}L#--kb9=OW!khZ*Rf5OGD{{^rYgnR^~5?ltY zf^{!;p)o^d*JWYAkwv;DiI|_dRQeizW2#>^c_V~RcbdH0tQO~h8t0_^o>oOUDymL4tZK8LnLfV;ZDBDw2B)3>u=aEqpCtItb6P?^WW$Dzct(?|bwuV@6=7 zKNg!v>KvLaZ4$bYs6^M30XRBF9@n;<*QTvCp>WjZ#?+^?<|}_oklXH{Z>iM`DNd^D zco(fuEpWmqCO3K(*A4P+tYk{LFsSATZ%+}vqBUZ^3xQnUhUO!xb^1sg8)eM5qjm--FQ z9Qk?H0SJw?0nQPy`$u)rIidC;8+d3vmoE(=fs!R{QH|71vTG?z|o5z{$=;!5v zL*dzq{KP}AtllZZQ{Fu~_@Q%ofr;3f>i#+v1X{e_*?}nfklqj69%xa+YRm zl1@czvzAq(LaNu7lJk%4@L5FK;pS}HfK}ZAZwGMFRZbkptQkH1y@^-P3-fYD9rvXP z%x0nf@)_LY6ANdM41`bW{$gnsZ0kkd_ZSSN*B=CSxk)rBoXu`+%kzas_Wy^yKMjYv{r~v!5~Wm>qLSsRP}Wjevs7fK2-#amh_MaEn2|~)MG~?M*|Sb| z!;C5W*4T}8B+D>kjBN}vhTnU*KHuZ`|L^^Nx38OyV~#QJ^L?JL_4#_7K{v7z8ou(bkp?j30sF!AzCCnw8=#D z`qQdl7EQ1sWcCl9t^6x_zF~HHi}dFkt4FfAsA^McDCA*knP=~{XNelF@Xj%M`){Q& z3Bj+x)R5X*6znB~Qb~LRQ-5AI4R*#SwK;AVvnBA^Tx-e8^hnBbB&5;yF;Y6;fpXAy zF9T5VqF=k7)=YARJWZ8>hop5RFz<9#khQ{(i)Xn`#+hYzKxSGug0s0`rK}{ycf+&p z56XMr^5j_*o?K;^6LV!ebf(*p8l@xj`)KzZxkBgWNGt`-(5;+oyonrq#7bKM{CS0D zV#Fo=u)W|%j8FARrG)Z!nu8dqzXW3k>o1=PeT!LeXFdOCg}B8OQ9!ctT@I*e;w`tf zph=W~TAozcmL~JE&c`!gBX>Q)M9c8)m~l9>u@*kviQV!Nn1nk(DM!);?N(bGM^V1* zk9_%On?>W>stzI(w(31q*rM;y59|!$mW|^NeD$5}-+2C&tDQI62V8^{3o3a{SZ-UC zHUIOD!l3p*@@6LKLgj)YIDghM|+c;g(Tp%p<&no0EGZ9vu z=VG@BVP53*r6*?Pj8`U0KT`W(kE~DaOyu%qv17(Ii@Uoi_Fm4O=ojNFOYAwd;fe(( z`V=sHz3f*%T%n^#EpmMT-yC#Z3qnTHYx8W7j%xU~>}I2^O{?RVdRH1 zD!(e^^6TxZD`J7RraN*30qC|Nf41IEw@FJG?p;|UsAr2i9{Ekq$I%`HsS&?Ikt=Gw z=j|gae1g+9Qf9CCsbX@{=o&$E3h6_M?7%StnOdzLzeFx&`P3+p3n3DA^^&!0PZ(e=w^EK_z71_Jf>=_acDV-$A3v%+|@5@Y~64s)tU>IECDvy?rnd+)M3Z z(6GZzF8vqFR$`L4?eF;DPchaYzc#aTpSwUSrmt3n?x$)5h196F!@=thS_3TMP>;FT z3HhP!Zjlfd>)>-Qw4QUb>pNpB*Ldo91Gk)^e{+Sh1@F-w2Vbx^6SOL?>-!cs+j}Qy zBBbN{C5bM7lR=A)1hw|$jsp&uT&{FkZPB!VM){afR6~zYVs?oyeZ`>Z!bbP?{Q3$Bz zgm3TIh((@X!r?QQ8$KK&h`_!X)N?Z02EK6DkyR0}w1pLsz3gw_%ZX-tdFtlszcT<@ zO~)6Z8b$b6`OCZ^?s$x@t~JUHGAN9<{OMxCYv(&&8c^A%g#T2&;vhOePCFCnU#%-r zGqR>i{4r7qiUF#j2){84M+BkR*L`C;9*iqkfKUNkLe}?#-4M_h_q6Sf-z?zoD!e~9 zS>uA{X6zNu%eKvsng9aAox?q2>>lp=UXKp#_#Eb`(Z<#nH(^uTM$hC>IMkA;D<8qS zl3th#%**8P)aVzIyM;O^BmUhE8w*SjfSpkHz-|GG>!d8QFKqM?C~7}n zl48iP;>_fSfLcicSy5Fsar#+?%5b2`Ga?I@f^*kM8)>}E!QGI}u4rLtfn_(>8%T*> zw9g`lXWPH~-VDAua@tua^I(7<^XuEsX#9MAY%X)E*DU|3z1kdmjyeB4Es@!3@6Vqu zL>ty-di7SP;@cwR)u{s2fRrrLJ>KD|^12O1pi5Qnwl|@2L>Yjuo)xUx7QHcUTk7)l z@`nAw?a+nyMx86!qvng_!;Rf>u$`WWD4{QUgX z*SlJKv*u{$aMPb_gMo8dRI4flXuy*{mYD>byMM2Ux{e+E)7FM_>aHl{kkZ) zms259E2yaXx}FA5{{Hqh30=b20bQ@)-CH6zT+^1$f39izqwa}}JoQ|3aCPCyGXZl!a>ZZj#sV~+zjev=MXaO13p00s6~A$#bmJ{W*EgF=Qo-t<>zM9I?8j8ph94=iX9imeuZ2yl+Oo zkZQyu5%d^oT0!Pt9fmk+3VJ|5(;-tWk2Z$OzGTSa+})``X1Qalr(>pnS3VttAE^kK zJ6rX$K(Zo2K>o7i5|h$P;yQHVn)BaGI@Fv z`QZpUar*qq>#T~iNRX%(!Q`?BBlDZ&LCURlb|Ke^k6qdAA-#kFbDGYismj0Gu*evT z8EqR=r!uf0QQM3>s|wDq9-bg=HjpAhvx~6TYK!o|PBx3sI9W)ClRJG94m!^z5z2a3ruTRDDavM2OX~@cRhvSB7JHn+4!#k)fTG9A zCS``9T<87HaCZR{%ZeTZvGK%#QNvg%SpeRU0fkipN&4uY1YhHy368d!Bc242JlV!M zxLXcyh2}lU;M*5*!Hl`lb<>Yx!DVzFGg39!M=2MK#ekmoD-Em4wv#YiIgO7z#2#)e ziyBEYvF+tE50c#@!~N5aY$-R73O#)1ufnUb`0oDYy{|?=9tKSYymFX*LXA=DJ-1QtZ~>21e)eEfovFN1>AbkGYs{96=6pv=^%FHW`m2{J1APLOtR>$6dJZ z`ep?%TQX9Y>zlgWsadI4DsW`}219aq{4#-YFQk4>!_D4mBaxi>F!Yu`icx`3q$CcF zUl()?Zq=tVqt2>0+2-Ku2-7+?Ak*Z_x6hhl1T%P*X9KV=QIdbB#QlZ<<##C14p8{a zHbrV>R8k808Dg~`pE}~Z4lLFPOe}E9{r2=L@dz2nHR}>RT-I^5NZf{(m|uZ)wTxnE z4(ck1=+u9$s}N+M!kjl-+qCQKMaT`ScIVb(h|y{IOMdo=k~MwT+=x!oelUQ6^;q;% zult@$_EvAUB)VV&)n>Mi`27%s$@t%rhZDhSv#^WCqKVt^x$STVdKE2BZS>9o__cN~ zg8z(*Fk|f+?9=ZJ<8-2?; zR57&YZJ%aMKef|$fMjK-Naz>U3&=5rGLc>wvK(d)6}GH`y8+&e58lLzD}I`XeKsH- zw7NtJq||XulKOvQ>>eU+n{2Xr&ePbc+ zuA|}bkDFI})7!WzB3Eh;KAnFCb2q$XTV&AB91BvlLx>jDcD)^0jinB&@V*_|yP|7a z@#g4hSNW-}WwPpAS8*0%s*v}#t=5Xemv@TXrenmGmEBBkrzJHEzALiRWB@X`C?J#b zCfZrsO7Vm(6)a<#WDU^X9v0MC{PVzF9xK8h?9H3|9m-g&TU zfggqtmnuZ^k&^>W`qv&#t2YlU1r@ISjgV@2a|CO+9g3OvSv6gGS~$&);OpfvqvJ=k z)$p8tw`OVfAooCve9eag;89%apeV5kYl0W(s{?m_9dM!TFYw1zWAI4+nqNW`*2^!| zo`_LnHy2R^r8kb|-hWh?S4g$Z;dZzC4SS=>>+blisS}l;Uf(lkdX)0q@{5O6*am2^ z^sL9t#qwx=io)%24zFtD#^A@lKt-hiy&K|G0Yb-CFl{%=Ne#K0gVoSoCdxk~*Gct@ zc!~b#ePJEb)S37;U2zcF1>igMSD}BNM+hux{EVAD{0~3+pS)w)*C8;mwz)UFLX)+R zSnOzh^M#2oN$f~$rCV5H zF=37ITq$sdvw^kwW@%w^_Da3Sc}TSeC7EGM&jO6}*nvG`0_MN#U&`8EKYAi^TgU@O z9y3*sqoo$jiMWFtZV6K7UAhjPW_z5T;-7PQ&6tmc5>4hg{;=-Kl&O}KUPT-!Imfc# zLF*09wfBhMt$A}=*Q^y}Q~OX~OiT-Wac|@0SOc}LlyklgMYJyda?yYlPrRH%70RE^agtz_!Z`?uo`QwsmA-JNHlHyc+GaTuQ%*efABJ@J-9w(EZW z(l-w-V)Wbm!H5H@F%H*6uKK@b58niozGNUQ;j1rOfRi4bQ0h==isCxCd{|lVELw6I z=S1aw*LSo|jeMA%8*4=#Fdj{k!6`yP7nE1T1z+<9^fu>y>hEx+vNg)98;#RLgUzc{HSJ*c$_gSz4-0rCv(qHe0)ooO|LNDA?Dk@blhj&F!<#B}Aw)9!@_N>> z<5g+0ii-umT&T21pl=;Dhl#;AMrqSpQW>qvQ(bk~8Ud-Z&lkQDocu)=S}PlG zr)O_#>pu7+cjQwCJ4gGH%Ri3>k{x00Bf}8To&cqh%#R%x`c;&+Vx%&74{o~J*!JRp ze1`$95<zwLPV1{PBX?8jPieLwv^2ocP{|G5i=d`2pLF_xicCjT zmfr%(G5Zd~<6`ayPWm~n6s!FPV1Vx+|5r}cxblDsrg?!s;slEvmRCU8^u3i-<9t#~ z_J>#iEBkpcJ4{&f(dg>BQd2|g`KU4e3L~@gE?AUSnK#M{>rr-+e!6#Nct~^mFFypb zxMzggcatFzopYwoM=<0qBq@mL-#IkVwPqzOzi~WxbKKZfI;)x*!*&n=_zr- ze)wo?6a4~BsA)(SyKEqfn7KjLFjQ_eaZH2dJoswTkN6sd^oS$5VF$Yzzis3FDlJ%+>+ zKJeG}xUU7>wo`LS@xr@d(lkLY>_xTlcbr73wjXtm->P;@v#%0ESNz$kf@CCD<}U{x zVLrVlEGYM9S~j1B`xpX?m}iR097a5UOTQ(A@i(o|i5&nb%o%21f#-}fwBq)#JgU?+_`~T^3Ax;7A95~PK-{!|} zZYJdx=D16f2qx46puxo?kCxw%d{jWtGi>kdhsY?!6SIYC*WQgIL!!DGM+7X&8;3kb zDh+_)WMG0_wI!hynyOkhO?xfi%)cNEg=t==6W<<&{<8kUTHU|2O&vix0RRL4;A_}N zXy$r7@3?*{JoF)~>wVqfs++~11b}fmo~yQ97jp;$N*RvcC`E&d$K~X^Gi)_c3)Mmt zzn6<~p_+i(3gtAhQWtSW1oX$oPKVoov;5{t*ktPSLZey*^s}nNGS+59GkxJ(#N@lL z78(BZgkIuM+yszf$Xrhi>`P7}9*4a%9G{y+W-xPYRvIl{g>{tYZZaW+se|uMXz3KR9WTqq0Eq;w}mV< z$pyg`Y3eQ_tkslMrnRn5!hULX*?1@QSC_MLJ?{E`7v)hG`IVnP6-?`~1lTJ7>!_La zw<99Jz^qo&ZeJ)qN0ys9rUko2Y7~5v@X%jHjji-h6f4>{A*JI&@VdF~Wg4)1B5NJS z_M4S9#iZyoLKv^07}J}s&ST3%{^_R3xu^r69KmnbH<5y8l8fLNmk#nn(JPuCaDI0h zLz-I4=3=G`hpvt-;Tm~CVI-r~trv^Hi3~D0FcGo=MxdDsH)=?d#*4*Htv_R?2v&{C zBh$})tA4)s)JxjDWO6vu-ty8^L2N%DV!=YKz-#tRdP(zf$q`Ll%2~xyF3O<9>+Z7IG?1@{6 z-F%THa|S0~-R)xtS7$ z)tENE^x(&jv1lNdK=kRo58y=%Ok#K;l3>_i0$xSbv>@v$Q3Ca*!Mz{ehnwl(U$7}u zS|xH$&{QeX^>=FMdy>Iq+5`Z;(BInRn!iXrGV6usOy1bsKT%)4MUv@dp7tD@@LXhI z96tdCL_|{G!~%~p^Y7SG$*ym*gOPxk;?iRcer(0^hG1$+-x;KD2TM*ecd==%c4qQs z8c;=h`~VOu^(enE0l~*?7C4AOBgh05?}&EpKCNxZEKH{2iR`>HrndB zFHU!7`I@3OvPb4KTg{tV(Ho#J>8H&W-@U>yn)#ql-Jg;OY9dNc^AG}Z?~3@7{|o^j z`YcZtsp9!#teS~9H}!QB<z3!&rUmF`r5%dm(DnQRWQl363{nTESm&B7_NzC^!3f+ zB}VQIFGHeI5FS|bOF`0tE*h}wi5dx%LcK&KcMp!K(1<0EVZ~SgkQ(s*E$c%CI^_p; zwa%-iz)|xp{J~S-wHQ?nq?hgAYV5GHIqoDc;zb^YlDCqb+BME55?bUGqnNm zq6)-RN1Ve~6mfy+k&jjN04koD=EMQQ;~n32`VKp#E~|csq(ux|9cC@J+axGM3V;E+ z;iF0(j|st#ZC>=^pg#NyBXz4!OQ~Xr)UX1`O^l zRveVUk~XO*=Q(#H2#pFOr+vJAx7y{7?(frJaAS$dBlFjMoBC(p<8A!H8qU{D#8gz(2K1K`V+w z6_lhASCg@{Ux6N6*UFe7YYY2~-kkLKIUWemM7?`=SBgzF*0AJK#-f>a>mt zBO)Y4CGG3Jmxtb;yVtnq-06b``&(6bg6%(IaB^4Gua{q~QSaY|h>VTRkw|%tvP^#( z+MXg#{+)Q;-ZLOYxyYgW@uwioWgA$!bX_Mx$<~wAd?YWiz$!n^1LzYTMYuzh&V5a; z4}{sBiC&Ebe>yepB>sN-!L;Q`I;r>LNHkVR#mdHl#!`A*oEA7)*tT*D$eUkY;G0rj z_>Jf%c7tkZiEPSjdWvO{17UnQIec;94lzU(p; zEG0zFhy^&jpO!$^4Akv=$wA5z;4ZkAVOg=M6Z0f+0lpGUe-W}JuP{TNG1guX-GCLi zAC88IMiX7!##=OI?0z{S-18a_^-C`&xA<+2yq)tXsq<}`h50yiR(Ijo{-0y~x`t%*Ye6oFtJTHt!oQoODG2PZ*Kr<~tG1?BCrvB&9 zs(Mc8PfYS#oaik){89(+eMD8h^zuAD(f^iDIEplt@(zIGy-K&-zvd8)jZzWKC&x}m zvt_b=dt>-tqpKfM8zsQ?F_ls^#9TRee#9Tr$&G)4%OZvx)T6@}ql*Dh;2Txb=+h(% z3S6By&y(dfaZ3SAic)As?KQ=m)AS0Q8QNBkx+ywci`>fjyDA%^xyn5-3M_c_JD5SJ zAUk({5y^fHx$z4Bng~9ly}cV%)ef7>5g=tmo0r-bNtec!`6F9xa?3=9P zjYW+>?$gRNMJ*`wy&08QB~dHQapDaL*!z_#KWabqL#V>+U+Ga%tk+DBshY1FZFwzx z^YX~ppG|q6qYL77x)tY%8D$6TgHfw$lSTpbmd^d@_WX;H$G)7Luz((Cp~bB6g->95 zl+(HH!lcBvLAhjMmsDsqFy4=6ZN$cpe{PG-Q#M@~nT9X?#UK^)VxM=aGdU8}K#{>K zl!kW=AM5!~8Ii(`2rZ~HAh_d-w>Nv!J&=E>Agc6nGWyDuq4340QamNYRsLW!!Wy^epk3IsyUBYtgQ2v8z_z`bi0O@?_9GDv||=u%kPg874f zrl|()>jzWglrHdxXHJ*)8+(vuDq8R@(dVba;#2&`myeg6 z=btcs{J89th_$Ix(F@fdCg*|`9y?D3LX@yNHrj@->5Su)J6iIQx-*o7vy<6Epjc0P zXq9pJjSw@?+36|l*j7}{mT>jkBsuE2lh{3e zGJTxc0K5NgSRZzAaiIpJ9MI|92H@C^oi+ZbB@HO)t&Xdmze~(0 z?+Bs@{ZuO-(|4Vyco0$R6!UWjzRZ+PrqMKkyJ7jwaO zU}7Q=Q>RF#F1vIdRh4#?ji%Oa4aq~BxD}U3&Zlp0RgRPnr$z#& z9`5j|fIL8W+ecD;%NV_h0nEHip>27zO*yHGXJ0P8j}}nA41{f};q+>!u9NAW5_gf~ z_=30)^1J`TUZyW$@(|6GGU(Rf* zQL7tShsUXc|0o=Ej5SYn&VNlljf=T+Mwtj;<5E7C&kg~)=F!)_pkF1xi@3l4sIuP? zM-S$BuW1!gkYq>v8?nEJ$Vn>oqLq2mREmOhB5smcEw8C9Vfem^x5=?%wyM(T{7ybb zivPtl8HIn0WAB+}hAlP`RE+;yCsLE${x3g6bB5W zZ93kba#p3@_@`CGo*;RUw^l#>ZE_WQV6KKf#nhfGte(3$R`5r>CTi<(CYnyLHLLVg zzh^2WWmWVt;KA{U(hKVVKDS3l4EwjrK*0gH*yDb-eoRvhPYHZA&462DRcVXlF7FNU zH}frM7GXIK9B~4?UHH`J+%>!QLUjS442Ov%)7=5#SW2wtck-#M7Gct-^8sIeRNcYlBZ)_P2 zOv(9((pXI&g|x>WQH-njWsxRwePg67!}##;Hie2?Rzi-VjFMpG`7SpqY2QBiB}UiL zq4qqUeyL+#(|NZgjX~(GPypB{m+yIU6wNEtmn4;ITlkcUL0|Zle3Y=(!m~txlE~S= zg-So;%%i!Hk;fo!)to%g4W+kd*W2t z3AuNnL^v2wc5rjr|7&1G-;eXU920r(EM>BFQo8?6jCNpoYNB#pbGFO25V9f=`3m>@ zzSX;+RgbAQoy zB9vmlKLi$32Ld_qPBHMM{6keydXPUrzA_w96h@0?!0oG_7mxa_T({1L=ih>hUWUd7 ziW=o-D$`n)pCMbw&2BZ7QJ&|}K^-*)ht~7qV1y;6#8FbOS$kbuQ(kWX>`uK+x*gfQ zjnsgyGiA5qX?{a3W^^^tf&83-&Fmc?_3zl5XQbJ{qVsVCLimZ3;roYLZxXtqE@$$x!o;17?^y^W{dY zs4ILl3F$hQjEMulFv-5q&G(b=Ypern{=^A~@qQ z+Gq$=qWNxYs=u4&>EPUIZ#jmcI*JmlOYI!~I=k_3AX24vspyLK+ArzP(a{qU=zOv$ z2~33rwDhGORX4u8{6-y#X5_<26hy)nu~KpY;vn0GVm(8+aFziOjS&B7UETE7hc{Yh^MJ&G|FIxhK)`*!jeQlScPBHfyNZxuR<$W%?`i zKm@);84;eF_bv3L2bC9G@VWjd`Kn8t8iZAk`YyU*P!Y^sRK6bq2*s4FRy&_88>nhJJ+I$1{p5_vbse^B-RQ zJIOm13p%f~NyW-nQ{D-DxUbS*+l){<+;2~iN^?+=@jt##6t=n-vW3Er=AF}QSv*b# zTQ+_~PiopIe9PM$tJKv4$GjpNo59)vriHpK3=VGtmJDSU=6Ioa9*S(4U*~ z-KP^)StC7_n|j_+Y1}W#PnGfYolx;nRa2dfF36}W4-q$&z2FwK54UPL6fwGjq=v&n ztQU8hb1zUpuA!uP2Kw)Wpcb6xE3Wzc2+Sb0S4SM7BbT0Dk?)Q;uX`k3iE#F~^(aeY zdEZPs65V1CR&z;f;mz)UJMMPUO8%z3J=N0m6pS<%;xZ!ZBU(NjI3=wx2l|-+OGd4U z1o%>h!Bf#8>IvY&2bf%qQ!v=9p|$Ap9n;t2(kdVutNes~URxj=-S>e0@#6N1#=4bK zN4`65$)USQcOCW)F-zoXt@5P6^bn=;r22J0Ox#wMn$y;$!so8__}co!zDxA!tMfQp*%>M>-tpiZ41%jKI!+93AsO9(DF^SjRoTc-& zit;<#QXvcaN&_w<>qK6u1Y?+nV+Z6tBMpC(IJ~6_X@>MLK z_tB(ft`8LU9dYcodcEy8znWe=OmP5~E- z50<7^tWv{uYx~|80aqV=9g(22$RKnNYjl*U(j5qO8;38To=p9PV>?o%uOecZoo546 zY$@Liao(xGwOX%`fR)zj;#!uDWTIAuy4pVmNQurbHK%+YE+^eNp?NVi^6Wp7wApF6 z24G~6hr~Y{X16sb8RjaeI1+*bNqwSI1SYj*c$E6#s&iN4+nMG7Wr+J#L{23H30NUF z+w?0}hKbudHHrzKM)3sU3t`bMRjf>-P=1SPUvn$QyjZe6w*qLH30O~3D!u)zlDujE zX_dTgsd-}mMu``8Tptstp`S=#nY2v~1ThtXG|-Fcv21Na4OudW?;51$DPd)Uxr zW23Ij_V|`6I);pmXOfpBHxqb*W|J4Xn??OVhwt-DH5t6_gdo~Lo25salz@8p0EdLP zU1dEaky~`=V(k3baKc&S_z#o5MlcAx)}o+-wGv$4UZ3O*5MNIQt}Q57FfePr7d5yl zA}Wwy5Tx0??3PN>?fXPBF{7U)a)i>qo}$6a;tdAOTUSUpdLYYEQeO&!u|qAJaBLgV zi&eTweK1p6p>9!zjtK$1nP59{S$vVb)gk4G6dBvv`@jX3 ziPEFVt4CzLwlgkAv5y;-Y;XIdMr9#t60Vo!EYtjTONT!`V@7gkT?X{4RWJL$c7FneB0( zsbi>Te$WblI>uS!h?`5eT%*oMeejQGu6y$z?mzMYF7KRs1E8~uu9U6-VXf-;ciWuW z@7n6r{W@61A8f7;cZrs20}y0;0rk95EQ$m|8a#}o8(fGt>d(3w|308&^tsZ1GE~`X&cg*H#&Wl7#p}q5pZ-Mvi1F>jT=qb_hkAv&YgHdc0Y*Qb`5S@~19`ePxJ$^=}u5;)2?G+S@JP`cinaJ+?Y5?Y9G^hG`lHTjireC}}AK}i7 zdeM52O0^x$2^Y(?UFcvKRcQ!B96O%egeK@#rtm{m>76;gS|LvUrjujC6 z{bv;ZKU&P8!%IK^#{&4zIREEb{`0&41tsk0^}mSizj$*en*T2t|1VSdA7Su6qwt?m z_|GT+RigjO7ys3EfC@DCf0e2K%69+NH2*8U{QnhS__CE#tS_1ye36yn;&Exi6+)fp z73u;X#CE$A+@);!EA8C>yB0@Bv+ODDYPQu8dl>ejH8@$>v$3o9;Eq4$;e7sa@}zrw zY!kvwpY}9%bc+5sD?%@ngX^eJF{Wm%b?u{?^9psu`PbC+@<^G5&%^D|`Zlhmj|#Dd zdWZ_$bw!WX&U~l#Vc}Zm!5!%CC!nnXv=u8a)RF&>%lzWtZg1q_E4^JT^DSw1ZzMwJ zUxxJ$trr_rT(9(Y8x#e6Ko5q$x$snw>JGvBVdx8EZ*(xNrw47T?mCLfJ|Sz}@XQam za_7&52KktwNr851gyO0GXe+hrH>$IfgTcDv^&jCgQ0vwDL*ok1cQEvNj%+E$i0$zS zFhB{{N<8G`^zIAN;Z7U=A|>U9pscmtp_7*`l1%Q=W-}4CYc0d#NnjlfWk_TDxIFTjTZ(-HS4(xfGl_BVi@(Op&39f{RN`WRepV2YX9t z7XsR>P~ab@Ff+{d?Y2?4r{edlENy-!R&jql=w(0F@e#wPcVKVKhO+k!;X84P=N9Al zcyG${)ZYH7xL4?JU|KrY^w&~bgNOxMjOq<6tIVu6$U85I=S$!FnER*07xiCeo?++R zAK4`+&j9_`cs4xT^}?rwgnwvi17G4K`(mVMZ4m~35%l3oXsS#S=FZdLVm-^>j-p8A z5rB8P=ieU7&!63|%Z^>NEzev<<@%;n227*D3XfVkFR6G&9}mmYsjTjqs!}y8vbj|F zWPT8B2xoa`b*%eDuPE11Y?ybs_j3N_0aBjza#`p8LYxvd;Uyw;V z$_hoc)F^yXar;h}k}c3ztN3t%rS;QMY-}0%Ah!pI@RFUWiSvTb zAdmN{Y}F}32;EMSbxP_*6@Hj1`y$WJ?ctYUs~icN3oL*Qf(tcb^^Sh}?tk-?*+N7QEY^TZfy@dIxdyHTOPAekRV*dO{DcOqhPD*psLG z%E_#u(7i9DHnvd{8EGqrJoJe>)vD(#8-UMU#5PZqBfA|ov7a0kLYAN4m7?;bNV*fY zkF-eL@&zb(cNBt4JaTzMNB=!mm85x5iBOzN*9!r;~=s zS<@k}yH_}xI2yd4sQPP*6>{}li44$a-#{0e5<&=pe}3+T&?Au?D_>lWPxLZUMLx1Y z^qk zXMdzb=0~m0Z8e)SXRXh7jwBUmug6Ia1Xz}Vl6uzU6H9K@1~Qa}jt|S9 z4AXyY8YTs1H{IvheQ7XM@81Xfrj<49?EFW4%sdfzLg>n{$MTl_B-?%wY=d?7ga!-j zKBETjnYX6Wp`pOVa8lrQA|xp$ntObHy}F0hvj*u| zZa4_Aew^z!PJ^>oHZ=G+Tk~Knz8es58vXg8R;@>CiY>Rv67jv}+kq_kQ{!ARq|(ZI z(g~**>ysikR6X*-RVVsj#&A%akR%Z6q!_vWktc*KU{~L^fnT+ zvdjugdREs}4;H@LCLkPJtFw|yhwA{%(I0tW@3GvS*mFbdfP~Zf#YW&K-M1BC`oPwI zbjE}6UJDo}V*A2saq78GcwpzF1&D#%oTg&-e*^E_d1JSX)i5jY>#lk?>B!QbF?w*e z(hkSLrEt{=*uq(66dx5O;kA{;w-Rk_jswDLaEIEP=Yk#Fd}Y`h;_J?ako^XSf z%s_Kh1hr2O(Wk#npc;vr@D=YlPGLu zqF(LpPKnVx+V?C+3viCpB}5Y%Wi-O-!zWD5=TvSgv`bki>INp)ZytZcB=>ewe^@z+ zU$FhxtKtYHikWV;L!YH|F`QOMQ<&d6SSAf?PuGj6Z z4VdjM%C}Yh;l~ro$;B2pEOAC(iqtv;c$!OESq7XvciC%$a?6@R>zYJLZ;l-(MR4WA z!)@g(L({X~w|tX?HnYxx1_m9S|B^SDPJp$0^6?y%t)ZDX#@;`w|FIR$dQj(2-5u(~`|t&23(`%$)VlH|%1lb6^atIPh0g}V!3t|rmv8Ko7L=scBt z0NODBDQLK@^p(bYl)9$k2{{M7{hGb_fRoUEIk5T2JKfs2ex3pM>!r_H&-tvqlGJPU zX`GDPp8a*O#I4o=<1L-Bha(OMB)r@8XaAL+Sn^yJtqcmBvrFXmvR60W|NiN@Pmg!E zN#m=3K}nGEhePiq~<7OSeNFkyMgZsXD5 zXw~#N9w&jeFDoJ%`UY0Zj;9^Ow=q5xy`j=4ps5A6wj3g}E6=$ha^Yxv1qXOofae79+eJbreWCE@%4?$oJ=rexRc}SMx;TkTZ^Om&;|p!fa4fvOcg(z!%(x77 zbje~=@h-3zw2l5Md+weXF(AI~XzmS|e})p~u(PUsa(0f3qjfi8j`_z_nENXaU-NIu zAphPG_F4e$PTzz&nVPII_c7IZ=BuNxy0&<`U`#~n{G@GX#Syh>nvmv-Yz*n+g}NM@ zuMA%P7QRa}GV9r%*YEK(0;=NzNerN=C1{CpvR5WPY&Y!U;|pC8)wFK2l$#z&1RE-~ zGXp-xzB<5<^)2(u`s=fmp*|^5{d?0V+9=4k8l=X~402EI`VqJ0*Ta-!&-yb${ZQYG zJB}7Ql-+zYrlc{mh9=xa8y!PUg}^37{D;49C~RP?z1wx9?~X&x67KASOQ>A zoZQR=Ip$G=c6>1MXR*7iIRLWLU`X-|~4;xc@AAd=THZ1y2-p9PHzs?Z`-Jip< z00ZuMk`!=nEU?6xa?~X_K-safeYi2nY3%dVrY+vXrKJ62OUEaJ@^*b%cI@bOoAUNi z3v53>8G}_PT@oBoqWEkJd)im_Zj2G6fH<47HDcrs^&{omRVJ&MIOoTG_USiJ**+R! zSlpvCX5T;FAe`jiEfRxfwL!%&_3WBtBgGxzDHwa=A(hbth(9KZN(w#%sz|agm z?=^pXo_C$K&iQn{xpXZS-q(HY{j0riUwPJ1*2T*iHD9%1co;x7H7gq{_qSaY4<#>p z3r766 zhSQv*W+?A+)^o?`yZmw$sDhXvT-x6p|6Ns28^haQ^ygw6b!DdvgWj>S$J8mnye`{g zruID~zpBM|4BXH?rgm8!72TV3reALK`(qce&x-W+Hg|Q{zM$Mfbe&k49)CJHb8CIS zwIGV^&ji}sr_o!J|Ah`gojnD2_OO8YyF>}Vs7cl;8z}vyDr|Oj?<^4GX5b_YmA2sKm1emabQ2xHqnOZ`wnYU zjY?LWZJ2YlrC5>*dC|&P5ckE;&m}u&%PJL2=mPQ{*4nN7iD~rBZ|`L^Wvd+_oT15y zI6dmzRwafI{n6)f)LQM8HLkX3RlfZ34LzpWwihq&bdhi+A@!_B&JXtnJ`s6!yHPBb z>oHyvcU}Jgt;$}kM9g$}-`J0bQ^X56Z(7}UnkhyHwAw=d=1_pQ&<0<5%^Z1aS zR?*gB+u|`C-DU;f_4r=lbZK#I)$ncT^RUf}4jkn$Y{eQ~u$H!8ZZHKwYJ6WTCuUt6 zcC-x;*a$RJE!NyzI(+Y(3m+)vQWV^TzX8BV@Pu$AZEFD*y-5)?eD5ZQa{l&8t{Z>< z9}%~RdlXu>p~(q^77A~pn?#KhYgzbP^Mbbk*P^@PT3PR(WRD8@8IDnL910JAn}FsdUoI$V z8k2S`-n-b|U>YG;-x1yTlQLvFsU-zbgE zj%m12PKmbv!5>-zInd)%C|!i82tbd&WkUP@Cf0Sr|KyJw+-}e4BfyLC`o3_YG;Jox zx(f9yb0tKTFSwOGuq@vtT`wHj{k4?iC7$fpd;TNk9bsDVY3-4!y;LFxbd7rn##io4&V@;ms$rmasx#&B(<^`vPM{-h*XRm&j#J^Zrwr!h4cR8TRD2Y8S-%i zbeU;O0-=AA)@5*+IcVP0kiEy8nTOkZN-TFy5UGU(ubw=ym%uIoz)Xi2GgcK0l$;%- zPgt>!91ee7ezp7NWAcS)8)b#0CoY39^a>T^5o*67%wCZ-IQn(rhQg}UaoFu?my~;NR8fBU7!b>tQj)vq;7Gi0_YJ|MnFMF0d zhT@nCdWI6RcoYB~&@(b*XdbgR)B%V`dr<>*Ap~=tx&P>>5qqN8BHE}}WXcRMM+%uM z=GbBGr?9$om@5E*ZiI~W$BQe=$#<)BvXF_)4pkPyeVvixwXSt`kN7YO3}%uCC@?j< zzs+i~Io)fZ#9RgpW?9lAd*%5tM1eBV8dmx$r|Czy*`QcHue2O!8s&2mySkygHD!wT|JJPSm24!4j7LWr25x z198gf2$VkxP%D4i!VLg_@HvhdvvCbLs7~3Rr=XjSMQ^D-+Ha2d=DK&OHzRc%xj)+3 zie67FBdMxS%57~nY*=@ks5Y%cMeU9~0R2;ZwRD18LymkbrF0Q+&PaQxLgbAz{0(C9 z{J@^?JAL@nkI62ww3^v=5n%d|;kXnj{`^N&U?SUxhZRi}mKjKzFFjAkd3bBHTTx2Y zCl7N*Qspe~6R05CBK4MTK;R#YIFpBesnAM#=-a11+fZjqJz4@E*b?ShUhN}$l}j|Q z=Dp}Q_&Tp8Cf>7_@VZ-1aZ*|sz341|Xdru%>9NhDe8+bOW>SqR!|npyPfE+X_R!iaxG@dLgXW(r1T zG1N{dgxW^g99)yE{j~uXY*qUIu1S^|dp3!m8>nq=FRW#N==5i|rTv@|0?m4nP?xkl zo6IO4l;C^Z>wX+;OXroAW+hjDG@>?GGID3EyXXwElxuq^WnZr9mO_W)s1btf#q^kv zM4ZW^&GEcic}>B1qIpQ&Wog2bDoITv=Z5dmK%tV|G4!lAvD#ieKlj+rFz%0i!H>ij|9FEJ>DsfR&t=0b&~OQy42h1&1yAo;lt^SmX`&T z1Oa0@sAT3y%jiR!uKiD6yJhQact=TWr%r!w*;|>US#_Z!E8CjdU7vq~ zM)P}g@I_o_Xcj-uq?T(|0m7vN+=dLLc+01*PZa~k1p9*A`}M6K9yPk@)W4DBwCtEE z1Z)KL?SG%Q0OC5NvmnYhOs|Bf7XTVPrK-(W@FeuXI@ccH!~d_(c|ev`d`gxx;jfTyk0a)lfH_gj%Gwu{+=5NwC5(^&0Is3&|vLK__4RZN3*beo_ZO+GbQkS7O^54k`DJ%sn1T81i6 z3W>Ij)_Lkd2*nJoj*ycOz&pYM1MjjJR0r(qSo^uvQdSr+7Km+o^?JTRH?>r>dZul| zgCi#ka@*LVY|f&1r%Zw9h4hX6eEZc2y_`HoEi^eHEi zwoX*oxzzwhp(oL?sHEQpi>IdXrn6)6Cf#w8)z^GCPb5LhVvXF0h-_;g4zs++W;s$m z>OOdSc5-9s;PU7>Vs8CBBCQfGcLo+ zVoH&i@rHIl7&_fU5;l2d&hkzv04;H;?D_nYg!{BIX8#R9xt5`l<>PPZpfgjbss<|x zP213!sUx|ZAFVG1v(emlOf9v_tq`{ETWx*&HUlykt4Hu3<`cW4e{^o4n(meoQ!oy* z$f|R>Fn?MZZ_SK2FBTD3nme}HkUQDT*;dFk2TM#lGOi>I$M3z!6oTw2SlmJNWlW5> z2HuvW-R1O*AtSn-1rnZMLId{rAb(8cZug5vE_P zvtn8qmEdF$rh8&kpW&cey!3Mr2};z^TMDdsV%lIDU531&SdMhXK52VjJazvSwz0qn zQQ@ODj`$6%%{K$w{V;_=3_f@2gX9;hq$;mraP~+A;?56@WZ_%9msVp%$-LQ6U zJxc)`!eLN@YLTRSQf#RD`h5FaM^o3ko{>+y!0%kHMWbabc#t`eD&l1HRw2Mh%sl9? zqmLHAfHTFDjJNJnM!aLZQ!CrTI^Ye;+PlIpqH~3|$Za(dqsVJw^X18jQag&UmsaKZ zbl$dW`H|`8y<0|IUr6-45%ME#JhtM7x%@fl8W&-Y@|J5+=G|YG_1;B+noV`a?|LcC z8>qwN&;kkZ{*Og?b)(Pj+LQ6-KQw74Z?qwU^~?7t2r|k zN`l-9>E9Y5HwTyf$7N^PRth4Exs*Z+KTSfs#u~n+Yepttw!VnvvwU92DW;L5)mtq& zh3-ADgU7On`@$N9om64D!V`TpeQ3_*ZSe#SHSrThmiDv<*wz@!fKR9&a#h)zF_r;4 zy`uOhCh(U0XN}RKuL$Ki4Zkjd()9&5f%Zg0@VXGJ2o!2wXmpMPssv3uxagVUF*#lP z?33d!vd9;g5b-VWgD7#AACP-3v9W3Wg+xQDaB3n*n8d^&*)U_l+;|f zq08%AAL!;5wlC#(LsuhDqgJ`0{AWat>R{#hd1trD-+l4ZDB0FFDNQEjgPWK-@(L|n z+iK&Iil!ccOE?pK@3s0=`f8qewJo}!H}`PYkd=Qgnp@YGjfwh z!M^n2LLJ9iiJxylgzn(Cols9bSvw}Qz4WTsuq5E#M5QQ8Q}DqV8X`ww3LpzQ|C7gO++LT8Vznntb-9w@ z9FDU%^B2K`-qn!j7S7`!}-PtC5Yn zfhC?m-xi>*XPW(Q01jyQPGE#B>3*6Ng$>bMO^pq`oG-zv;49mJ($n%eEg?8dI^Bqu zv?5r1?e>vn`bDo-j6SWi>+1~bbk;~sq^g#9?FMMEx`cOp6_j1N?#>v8o4mt1pd`M? z;j*HV{bz*iX2`D#-!(=aRMj3cFwmnaJ^w=WMy;b!$aA?PJOs zB3G~ETBoY))`PvhjJ><+Fh98bh3CoAfSpO}bh%G!sx3k&JF#p{lWT)-yGM$s^OopH z`$?_#-zm|RF~z*ft~1w=aBsXzPm#T{OyrNFr)<5&k3*008kWh#Cxm704fvjS>BOiM zm?W>|t(H?|G^y||A}efKl-w`c`WB+(F^r&a>gNBRYPR^?27WbMerAHN+r3Pw#OPvs zlXZk#Gf-y>q$-w1ti*$=6Yl|vFZ7S%13)xE?KeB-9OgllyQ=ebm}=43Foy0_OF}L$j&OX~kiWUc;}RyvmcPrxOqDc$08?6A?x$Er|#w@3%DAdzfWGt1`lZO#l21 z1{r|B94K>e{G>3R&^J<3LR>I02_hKONpO4*a!PQvAKLt*XbBju$1fg_fVGP8%z473 zaPKF2KBv#Lm2H$52xqGSFF6NTG4Kz&>Y%l$l1=(T)MWn7Bsd}kF; zORT`x6r82mUe{YN3NBVGrrC5I&6Z49fxT4F(t0w)*C!0k@l1DCpXF^Y-8^t`z&sS6q)ZiW=xn;` zUtfNs23|=s0p%Kg_Z*EbF3-#1DzuMb&7Sc@siV=fAi^#o3o`0A9dxH)mhuOak6E7g#j@Y*F+ zpq4V-lp@GAT>MW_CeD6c8kn&E1|Lr9wVUtwHU2qPqFqLBn;5@yRE5|he6Y-CJ^j@S^*H+|3aPjFP5;yd1W#3H(F?95Xu{6wyn3?QLj3mg zj(?CTJ5m>GFq?&D@c2e6)1aAFT=;1JzZvKMZPS)l1$a0ZubiGh5im~0&nWOo=oNva zK&j)yZocm5z6)j`QQe#7`Uy6=Y_!y(&%3HFM#wcHF43FhuD&xQtB4BW7pK$xg0|kB zZZBL>@>BAqi-_M%Bd}&#_6326}FxAEQT?K z;rcci@7fJdoS*Tz??-XHqf$a#JXlV^iAEzD=1OwkpRVETflgDtp`iH~Bfwez@5NNN z)70txfE?ZRWL!*V)xCfSh8$ZEuM&zY8y&wbGjIo6w^aWbD8%@;NAwr6XFm z7Dz{c?xU8Go-K9k231C11hFUXde*(OuXLptRsuL^RnHc6B)YdZk__7Hk@WwwL_S;7 zmh>=mEVH}w1#Oj@pHSIb8Q9szxmp>j3E|zDIc#NM7T zDbG!&+z1rPpe`u4IuFUH(*KX~t1a~n8)1~S1e^uNH& zKE@C(H-52&D@8;bETc9r3P#5TgIAddea1#-N_N^wO4h!TC~=>E(Jz=y$c1w!O1VY8 zK5d>OtTtq<{aQL_z9?|V#|J%c9ediz1)J$%k6_WNbBZHoe%)A^;PIluL)C@b&>(SX zgw#GvORL9-i$+x7F%s+$4p2S{?_?VRt#QWpo&FI1HU(D$*CatDCC=r8ew|i>)tfOT z8uDEyA#KVGG=QF>w&#pEo}4MoV1GEdlhtZKV#rT;xpuuLqa{btn<6|v-g5Xk#*v9W z=ei8yrJ3eY8a=NVs5I!D#)vo}%KxM_89$I7iC*NgMojtRBOf8o-%c*+wV4=SX?Evm z0{nWQN^OY1K7+NGOaENheV0H1-BUb}j+446KBZMVk5RZk!#P}Oz%e0`tj|nI*=c$8 zG$W6)o?)Pt+0+WE<;uATKmTE3gK!Klz5P6XJdc2K+*bk!3o4o?CO$knvq;qp+mjfL zT#6f@#4pB7O?;FX2hj(&@t!sDt40TOH!qPynUIRCJdh+Ba$b7Rk1_l(F+Pkp94zq+ zG`eRbvtYGf{GuD!DN66lbF@9hRoZsR3Wo`^jj;*JjK)uGJ3pA$;?n18K_>s2lB^a~ zVjb(;<^?V^xBZ$^1D$Gvr1QG70R+z7bynTf^78&E>CmiphN1f-H)T}SvQd>+z~UqJF#Fx5PeIi{s!NoW z{2Wdo=ljJee|4}5j^Wr1 z^386RTNgu~D!oBG(QoPB?ue5`i69p|Vm1RQ4|xBC&U#&XcKWe46r+c-wD^Wn#UD~v zY9ha>I&8p0oT*2a#jAINvuYhA;0;u4<$4;2F2b8uV1?Hid}4&Z$*8~<-Mq}uOvA@q z=hv7FG;B(dEA~JD!jPaS3cHkvcw>qp%vh$I@D(7w0GA|n5vJH;rvei>t#*eK>n$0$ zXAm~R*)uB8Si-$dk1Ih-j5pb< zyG%Tz*h#1~!!FhYs*n^K`dI6pAU%>5ke#CpYyIkxN*g{%bBKq7!(jLTm-i_#i)KwW zG^f(O2TEZI++Sq)waf8ZxMw_+5X&<1BJEbiPLJu%4MnNOu-k@1!RfS)3on<{JfTT@v>u50hIb|DNFnU}BA5sqbC;C*5fS;7E>7 zt*ESrk?ywo2R>k{vR+AdPOq{l{jz~*n$2GKU1AN#I(U&4uk!c4&buYH(${`fcmtu! zjy_OL=;cx|)b!B*o*@77NuLmK3X9$LJWCyBD!QDzJAte-rx&6`cfLzj5gG;I+bJ*~ zq!I5H+sMVwVh<1Us8(5)qc^X&N*M`QV5I6Gt^}jK%0_w z4Hqgw2w@=VneIO30do%aJp_8l+hV&|DLzs*4Ji!J*yhCi!pciOqLokukcqsByiYm$ z_aleqQIkdQe%3jjJ#3(po%zp?koe!CImwat3f@4oI^{orJ~IRqb{(RW6#Krw zOo1_EJW=bIZ1*TUXjRlB%rRVIJ*L)H8>TZ`y_om^uO9k6X|s=1(4BTqEth*zFqTp? zEM8I*!K}xgV)O-6eu$ONR7}5ECucse=eBAwCIIQuX}?fsGI%`RwwvtEQ`GZQNv>!? z=~1q!1wE#Lpnv)*vUX}bIt!CW@ai5yU%0Xk^Yer4GI>;eFX$ls>dX#fs5u=IEVC*<%z^x%KPHVkzGk81P29_-hr zSpGvMO#0>P_c6kY_Wxr1{%Sq@1jjHQ{$AN&S+W{@ByCU4@b_v22s18CJf~@$?!5`Z zM~VN!NBq>ofgcMcp02(d0mcm#igV3wHumw5xY3%P&6Rn7PGf#}8kA--^~C^o7-L;` z+)$rkQaw(F71d`zPa}R_E<3XMNrD-(tdaT68pBmjivmI4N~bsLn*TRfx)gs^BkfvR z{Uq5XDDijahMcjzfZzA^!!W#qT8o-xIk(lYh&9Z9v5RBx$i}^L!EULzetYOC;pN{m zrC(i2gHSjk(n>tiKo50HBI~9H;QcF%X^KR(6bW)KF9P%ORRC)Fdrh%!+K)kWvO$_H zkGEPum8k2&J786DPt?!&5{e8Nr@w0;S7fWcvpQ;_*TWww^r(GQHb5M4Nt8g+dw~{( z`3#u@HqdMGj3qC)sqZKIKK@=*3L(jG|7%**5c^B9#xh5oj$Er4MMX+ zWIi#QEp7RI48iUB?CpR|JAT}u$9hi8yd@&N)C}=B;Zua@^j7*mr0_IwEgRl`P2Z+z z@9#7_`-r&ov9$zkz`uJn&vo12e!2wszX_2&nLMkoNp!3{_H;dkX3y5?8LablG$j2` z+`yAPJSbY!H;$Oaka?x>j;}VSk;uN}-!1UdHbKi0*)%)2TAu0WL<)40-ZYu1AxySs zUC5z~!f*I0d7VQ=WGO=%2TY8(+uI=BY%j9l+NiT$X^J{^IDctFKO1;appvl@^Xl`@L*9B>$OARkbH+30(Hjss#?@V> zV#D|`+i-sL(aZdvI^c=25MQ;c+qVL8313P4elxECsYqtpAhaqr^mIXnM;D$8fHcg- zX)is2r@>ffjL!2+O%DrD2Y35yf<4_p1$VH~eUa`Uw6+~SB?$%Tv1 zmx!AuJF~km8)Lwk(xtCBQ_iD1mMFCp= z$Q*9Fg$Ho%@3!n9E~dZZrVO9)Iqq<0fa9WN_D(c0*%2~Kb|kzXkROb}nh)-S?cvT~ z^))bvAlmpza)_V>HYBgne;zqKfo_tC)|7T_k(Si zI8R{qED8+8c)OR_ zBo2Hlx&}-2Y@+$f?QteySM0@LVmO?DkM2#VM4fPah0XY5DE!!6)iWQNuG)U+$&mz; zlNNnC0mYpH8thA~Ori_$A*s&7#6P>l_wJ0){pRS;m#10>MAPo%@mNpt_`1I63T1pbI|;}N}_swH8g>$?>!Thk{v68 z)b2LRi%r`GUz9*vq}98z4$^;c8Mo%6-0#I2kA9i&M-$AA&L9F2Dfal6@bw|Y#-H8k z>Qa6zCNN|`#C!C-(f0a~0L*2JJn!vo5)Nii*>blxmT$jVdAMrPYq!)llmJ?RPrsUX zI!Nll8if1qlQZ2qI?K0QsdYUFa-O1k4EzyF@5SWpUTRdHg4>okoI4+PyF0DESgQJ6 zYEoT)os|KLK~BJtPYq6kyXO}Ln)=t3ZVugo$*V`T^?4vv5C*<8?U9!c0)kNj{MN)= zYU&{T)?4WTD!(K3#$yu8h~0wK!+5c98$@{g9TKkVM&>V%r4uc?WVM_RY|ah_?kvSu znxBFz;MQa;nxPOG2CIdAIKiE$Ji~``cTWCCRC|8>x1^0W8He*!nTtiEN&L4Mh=xrx z*!d3X>ZX`I5?pPfh8?aa=>PZX5zE8D_Rku+9tCo4Tw>TpaQ7A^t zOhs2OM==@i{)}LobmWH_5KehoLG5vNhHu_Itn9OURj3A}%U>nR{m@F@*UHymLiRv} zZmE19w!6HzGf`vvRcSXpYJI$+n$lfYElmgotX!+a)_%YTdrs_Xt#!*4`3fglr}zZE zuMwiYJlkG57sKB}dt(~>Vbo%}=1*3b&Ba`jy^?LKG^S$IfzT>!5g*KEx4oYJ%=6v=vfa+H@16$36TXJ?>7z2lrbR9@bQF&AXm&{2gV$ z7oxepTU!s6rcip?TEPvZ#V_jFPwhWn1$jWKMv35fPL$X9c>!(=?pppzA+<#oGUCY7 zqComXaVdF|-?v9uMxV*2lrGMSRr5?W;9*=w7de}=u=dKk6k9~hzucMeU^6U1iP*=j z+|dlj*`L6~hC1o&e^YuAjh29!YT|Px-YEi|<%WpBvwL93VXnqQtjSRu`z~lJH{Wmi ze_Y0#*p2+sAIF6SwabxKGHrznpY;Zx62ef2iRp%C8!*;wxag?j%_EM8IqAeDlf*ys zb#eh(Y?=6{b~JFqOW6BQUT$zVIQMK@on@t~$Po0=qerSKW)`lzhxBpkVH-*%jv%`l zl&c3=Z=LJdxHaCAGznvoT2phZ!^zLA6DD^V%`DOtjT67_)%-{h8lsYO3t=Z%JTqZz zcBPp%jC0PRSY-Y>OO4u4?lm9!>D|pxQv5!Lj{7(k2CH3T2#e0qf5GWi7i}k+w*rk0 zp-z~?fOpH?8Y@G@cCDUaJ5IO&a?0-;2xiN?+2O!?PTDWT;$@zBetzPg?z{*FJp}w} zO6TPe|e1H54Q1m$_PfzW_UUhb*yaxsNeajBU+7C4#@XQ^nR)xiz!)`oZ}h zMi-Z7c=dFIVtC)w|C`JFC=rJT%<9tXY1O0PR?Wnvq|@UvC<1UMo0d+8t;#7usVw;P zmIUn2Ow#jnMz z>TuHTs%YP?UE3p?TK#@$-NPLtQ{TO_Qj^J!lL^ugOwBSpY#Smy^_qFCgV|~vC@Vj#VsYiZ>V{q{cR^HVPhU?0 z1m7uSf9X2(@-v((8V{-%E-H+2x3ln7)TFEHy+>#1_k1}fKU((!{ii$Rpa8H|F)(AO zf7tcN9^r1=|3T30iMwxd9&bMF#r8p;3`kOz)}8f1y{4l624N`8@&8a7(*El1_y5vT zVS}oX(=N;1S!YbuIqfBP>th)z(?6==VconsJlktMoVe+)em4#9rmv33oQ+;~l_z%< z7F_1#@~^E8I|liRf`BBO_}zGfC;Lp026;rZpQ|&9L(KCWika60@yGYc5?MTYa`8#` zrWuai!=on0ni!70NgXr1QziKa;V=#|W|Be|0va0POT_)hzpQ}Nup0O4HbDmukn56E z9BHu!rD>@a+%TDOw?}A=6pg|j8c!5tq~3ZY?7x)%XwLqG@wD9C+v&!1UbSLD8T&dM z-k!C4(azlGDc^yTFp=$0ewkc<_R%qQ{D#0~7TmsZAk^hxHX|F?Use4zHn9|eid6Ap zDcdNDz0Z8xV3U@W&Lx!Y_P4?GoU%03>{)%RDF0HhN$G@%WBgKOa!9m^t4o)we|#b^ z^qlEEzD_T<2scD;Azbw`ONnrypW?|B^lf96vxA(K^H>d_qOIIICD_ep^u(;Hua#=6 zil1#+d-Hs*?%Osx&naBxdL~|TIqB&pW6bB7=YO9@gbpqu+!KQ{%bGY+VP-xSb#x<6 zkC9MVS>U~wrsBHWNoUl_nhwHMw*Qk zzpPZ({g0-%a)OkC7gP40RTAOa$?g1*pv1lFv7*mn$kbDkNs~<@Ed1hpy!h{2auBbs zW(2ZDsZyclP3mn2&#$!n@(g@*bEzN0X>j%3T#F?;gpfD~iB*7Jlv=m%=^#w@e^5nP zY}$$xC>vY2`*?RRaTy-vEW6Y{&4}?}80#JlSsHXTZ^%hncn~D~P57?mV@08nlKg0~ z$A)5_iKgUyOi>?wypmww9cGVB_g4F`o4#%sV9iV76;ok)=QY1SL~~vuyQdMPts&PI zH+F}#hKc>Oai0`Y-!biyp!L9_Y)>O?fdAns?Ek*#WD@IffJD}Ek$TE$+t`$a@|VXo~ykL-uW zqAo~!f>O+wLq^eD9<$D0)Gf4UOEz4v=ACb9iNFm8jgV7^ zjh8_j-GdZcH=7Z74-TciYyP~vU&pfnZ6QebKf# z;;#|`;D*v;o48^>P{5pHVxcADy-BUg2=Ov>QJwdqmc!Do!@$|7*khYvBGUCNnXhrx zZat}r+(!*+=SP7vQ*L(y#)J%@HuTQQ-9-UEGm*&OI#spC0dKLP zIp?#4RZ^^EjC8A;c3n z@oW5y`qnWah=*MRAJnvb1~r)&d>Wz<GElrY7Y0>x})Uf{n)2wDd%E*9wD10`NS0a z>Js?pW$OiI{@dz9$NW#X7CUbG?Ug-!%9$W>{P2?+onqS~b-&?C<6UI273yY8TFKkyfI=*`&;tlrvXr<~ckR8t+HwWIRK#>;D= z#!>4BZE_o@_I4CwAzQsql4~15nwH$}EXB65|IA&GKnzstoXINx8kc0w3^tPJcNxkF z&$tGUKZ5vY?(xUoYCK1;QT~oPv4fwKi0RW`%B^hr=-$s!6L$KUOcS0R7qUlAjGH%o zqg5|><{DD$NC(0xf_>ijTN@BvDy*K|Sqi7Fti%`mS^wRfpTIC{Y&XV~C*N4=Xj_QZ zE)R0UgvwEjC(R%W-p(2J$e1)%@q1@p_+r(}4?=3cq&KeEA z?qAqXoSnzZD?Cxd5PYRyLEV*K6UBwW#K8e}!nwp;v**EiC(BcuO-gfoq%b?p>`CJ0 zlzTs+f1+`LPxDW)fqUjm-|lRStATKpu?{40`>X2(?P=(ndG^y@6by}IIAxfz9DqdH z8KO+3lrxLFl5J)QnN^<&Iu{2iW6cXhrhq@Xi*{Wke3 z5rgH@i*k+%%=?Nb?|{AV8XUL4=Cj;{1911lFWIhL!xHmz69}G4JkjCbs3jZzJKOLz zX=3UY!CVkbX~^PL-ZecXeG-c&+usEzjfkLrJE!F9BTyq+k5TWumqzqd&hU^D?-|=X zljoHqc3O%;Ml(WdE{-jGI#$c^tC{gM97fI(sO3JFQUzn%^MzfO5|4SNedCU?W=p!U zOY|1`llLP1y@vM##!6QZ6Z-{GB#vX1NQL;N*`T+iyRQ$=XB5iS)en5DeDe5MTZoBL<{bkeV%siM>C0E^f64- zy^}#IJ{L#C3vpJZ)!P%v_Dfas1uPD4ZcYyRNbKtiCLG%jL@2}_ujCW)k87u)Z8qXL z2ib`4#GcmcHAEW3qrNDR^>=F$xzm@On;C)oKW(bS9+s537)qmp%FMtbHf&05b2 zjbsuQ7vXK$_)M|lQ~aA@DlL$2 zVt%*HExgR`t-HNkYj5E=l}((kBbn_~JR#3W-*(61%^(@N4}Q0UYb)u<@pn3QrT_c{K)QEUbs#8*HomQaVY5yN%f)gOLOY zzL{U|ZC|WLWqZ;Aq?UwR|B8a(TQGoc1NIDL*zZ^x?M0#{BR+qu#93Gt@aar5NLPq- z(u?u<;v%Z%S@rXNzpamL;;*K1Gg!Q)b@TvA`H6Vj`G}H8Skhyoz6hH2cjiZ1AFver z9**nBx#Uol=zhhg4hnL-CzALuDmumHMa6S~13n&qh%O6bVZHqQ zGo6_2UA2_}T(&#V+O~{GxO#{W0Xfw_Z75|dFv%^L3;?hil}Tt&bZLmrG!sEDe$8+= zZY1okS<4c4s>-hh_G2@*0WxSFyE%)Li%tvXdBHF`9kuS&L6fBy9ckilpkmi*M6Wyo1y*IBF%peQQq_V`A+Nxo)mVI@VX7w zd}4Mn{#DV~RQ%QY20E9mdH?gUhlQU%45CP)vcpd5%FMo&^r3*QE{?KPjb9XR)|Zp8@S_R z@=&+J+U3oO%rQUFTgzwBX5(=NqP~7!gU9*0sW-N^*ysP=v|AxlVvm`Xx#gkk=9sw& zcU}k|=vG&ZHZ!o2Z@JI=chjc91N4cPI1v2eM+qXx@5!bk=Rp-y1viqrC;G ze+@S3w>V6ldg^=5t<&I?w>e!T<6<|(^-}c?IU1isBPU?DzJlkIhl{Ns`U@bEsIs$o z%`Af__OU=>m2jez&VJd~2w$G$1N)cOXx$RWIy`+|7bhPCnxpQsoOgu4r|AM%W;N7a%)&=p{5}oAiQgjHt>fTiK&U2 zt-5=LOyI@gVIs!>i2Klfm=|R|FbSK(axwqR|46e3PTSy9^SUB;acdw=XT@?+BY6%) z;HoSqb25qLie0A-pP7EXU6dS4Pg6@Hr^WgBg658BT74{h!6MIOCEB+$_^@^aZ*I2+ zAv_rGRLBC!YcHE9KrIoLn)n69Z~rZEcUG3b&n0G8ZPaQE{Ul?GMiSl1_rhy-8+!Zt zPL8Lp#-emLBM+`W{wGcuRjhf(4?EYd$YMHx93m_=vCzpyLC-{xJ$rtXEul#NPzMK=m67#I zH%C1u7fal#ypa9#58}IQ@VdB@v4*f+N?hoaO2%9Fb@#DHmx?7`4J?lF3hrq9i7M-O z6H}^&n|ZE-rF;kOs1>K_Q+f(d9skOpx|B9HMAFF?*P{1qoS7Qo;uZlt)wC zBPZ`Fznw|co_Xy=$vxfdo_jSHn%)m>Y3s7lSt#%9@U4%|iRkTQw1tQu!)u8;*kd{P zPU_hZlH2(2ixcn7WS4Lm-b4&ixlQm7sVMw@er7qio;&(>%rd#gic9`NS(R&k+S`=$ z;L--(*{j%uzhWk2NGQ?heH|iBP_={fX?XqO9j+6_sOHRd1D4IUwZ+ZC9?$e5@{g>B z71>*~4yClB-RB*(zJY5Dm3cyM3n<#Y2kTTMF&c8=jC8tX%aq2R#XvpasLMl};5QzQ z6wvuQ0m2_GqJH0M`Vqza?(cY^-H#{HHdd~WLcE9IeXmFpQFCN{JZAnzPoDCG<%jtg zVa;EG63oozgWDg(k^&Ny!bMpct7;&%pXDACsDa94+@<&yS4!jKqMN#4=Uu-l;%`}t zRY`6L44u9_KiMDFH4-MzNs_9!l0R~8tjkrXyVY#PFM6jTe4-zP9ETRtZ1ehC$mcd5 zH0K;KDerIRL6c)yV;x7&pd}ZuCbfvVBX_@1NT6 z2`kHR_!UT$5tPOSdA|GM@uxd8CQ{}(&=vmSkOIYtA0Js>7k#+Cem1A0I=PVEra)1k z<-M?XNd@mrDmbI>Qtb;ILL~7ti#56EX2T@ChwF)k{}Q_ke_ScMG@OFI?qkp5m1<;f;*_7q}`MmM#Z4I!@uZmA0rA|(sUFN zEx|1Qr^^*bP_vV#N9csQ{iYK8o?O%21XVgko)Rb~<>p@1zA+#V47f?y|KqK>Ma zZIr`GzqC8=jhpw@?&2BbThbF2XtWeAXe=SOM7;~Qbf(ixVgFQ;;8He?;;OrD=hp3K zUnh6NZ$aexYWs@3RQ<+o%sW%`QF@g;+tB7wcT9sB+=@iVS~!Xe(r)13bQCS094?oM z1>LazzQqdh+o}1&(U%9wI<#4kGd;XVO*4D=N&h@qNd-S(jKp>@Cu$8+-0pF*)4LrM z|LC1Tp-vLGh1GW1TK|-?kUI&5SYf&edSi#WK8FYDI#0(YWUiVq;!!tX9p^#xj0d=)@%9nj%=+>}-xDR6StSzq=hUZ_o zNW!Wp%S$k_FfFuUfgQR0EiQ$AI^(y;1HakrZBWi9{yM{El(&Xn<>tIOCTxN2f&)+BWdZ3o9W6Vw61e&hE~WLgdzEiVJQP=Jv(c zECoqeV~(ROR+V5URYHWioo$RO%YR<@Pl8&X>iI+iq`Pf->Ybs>Z!Dt61csIJeO|Lb zHDyuZQE}H>Yu{JIV0Qj-^TH)r9;>}$Nb`#bhl@YgM1R)nbtl-7eGRm7BFBXy^b6Ky z2QQ*3-u4CBju>7r1gW4dnS~x+hRM`cJ-~%_+Z8TgO&SdT&1F@Y%Crk4KZCQB_;ikD zFRwGuV*ni3;daICwC4i-dNM4cCksb*rdG4^K0?ecobJU05`JL50%}z6O7#YBr$;J8 zis2`&TRy4tp`5KkU$Erg!&` zhGpLDC(V5{RtcS=a>>kKCE-6B3Q6BLI#9shy7?@U|V8>?a>TiSyGXKd<#Jl@f-b_u&+V5e}X^@ty*Vg{UM=_vv4S2 zSp8gZ-N#XnB|(=tm%532s@^$yO@QWt;VqQ?J9sRjhrHvqx#B$JS|1`^C)ZB6&+?cJ zqPro3`@KBraxHr*dHW^su?`eh-|Oe6I(US*(6_2T$U+S*UCy|lx>L0=<$d+&y{SdIx%*xLF|LhKnbmCiL%Ojrox%N0LFP1?m~1Aj zX)BXuGuk>%T1lL}WU^(6%juh^F;7Qy zH^;^@A)WZpOD1{q{9QLh4mDQiKcO^yzxG8QJrVP76RJbZ^MhQR?y} zjC9L4CNkFcAkBIy8BqkUa$tF!K5V+GubJ(-<+!zvY~x~K97RPWir zA~(f}Ir(lqtI`QxTy|$i56w zDqEx^YuR^Eb~930vS-gSWM9WNmKig1pIP3^=l%VD-uLhR^M2g-KRp`5>zs3)>w2E& zwVdnqAU30^ATQtris&6NRoD0Zj;*_f%k-IyCyq>qZ9SYZrpEwxg@qka;F3uUrZ4Gpd8b*mge@d-*AE>8=SM8*d@kK49_sn6ILf;;@cY+O50Iwts(sJo z0ZACmt;>*N>IP0ru21g^*avoz8b_5X?{{&ErFZphCb~&E0cm0w*}5WWHFh30N-H$) z3jW8HY!1GQcU|2DwqY)tpNdPh_QzIzuQHYx8=op)BAc^@z(y`Xb2l_0M9~PQ zk>;n<&zl}$yA@Nj%1-|70o3M4xLu@*lxpsm`%!3<^aq&ZKU4H1oL?s!kyHgKGdQ%u zGh>z-qLE>A-v=-A%nu2*`|Q%Z07dvcojukne#&YxZO&wvwDRH-Fb_C3ujME5(;+SG#-2)ig8l6Ug(N zvKr#lzB~KQ+|@1kW_e8}OR73?=BF7+gKa&n>?`Zu#>ZMQA8?Odorgv3LPL{7RzpA= z&)4!m<>HEthS}*D2=9Y%sqI%?qu*bIsIe#rg}3@9cQ;To2+ON-5|@AnfZo9-k^ms< z+pONrK-d}oi%MbX2c?dEzG*099iR8Jz`NkMX&&P9t>i6I;}sJ%FY_?o5DB**$)T@j zqhDF9KPouwGn3&VquFdNG!5GQWtoxhyEWeDzSa=+9NAKM^*~E4Xj7ckN1nYrT4eWB z;8vd4i7xovZvRpT)k`XFrmH!VN)Ys>qD#tsjfR0(_(o2^6Z}P6ZC`U?6B99wE)7{O za;IA81+`4TeMav#X2pwjc7z_v6n$~oSuPk|{Ie)5sN%>kl<}(G@8}j|;p_?IZnyQf z=~9T~s82ID5LqP|CqOi~#2mfXFEJTiN6^?iXzeWi{`ixMxoL-qI=}qUgpEEqev(Mw zCtY#f4`%NUp3Wh_eOh!K`H8LqQB}%&1LIyp4gD97hvXrzx!k{(Nl5a>>rUA9=M!vq zlyjcT@4fAnXngz;6)mB3v;zAFMj_IZt-&Ra{+_x($V8y>G z=q{yYN1@}Us|m4(8~8-olc*zgx8<+jWi~tTaCwJ62QJTrl~*hi7ISi1H_eW{eaA^p zzhvW>565P11h51j)PC*xH`g}FGkhmC%`p0=BL zSqE|>d{n8l*ojnFs~6aDB`xcDsjmL8bym5rQLp>?A~d>G(;PWa!34t#M&HdH_{kHsR+{wBy>rCGC^?FH{LE~1HTr94)K(jcf zso&3l5q>%a%TOdty4TVaY8FMf1P zJ4mm(s}!l;q@S}O$HFF31cI0i1H@0XbM;wokqP(F(AD7 zD=3#CDno{-ij^}DgEjlZrmu*mT*K|uML*s3IPsI2p+s*Kb8gyme`dBo&XgJ@x}^4- z`mHQ|TR_~T?K>Bg8g_vrZs^g!b>etK-@}@5J(x~_1 zGw+9W>xFJa^0OPu&ml!{kTz(bDdnaiiVu@!19x%#zsl)ABX>&3pBkjl@T3|(2w zq1Ps?-*?lM2Icrn{^zRrsU;^?zou9U-$6D+rj~}UiEycv?lGln&}DtVa1VU8u?3gb zfI?QXgXkktRl|p_dkT`(j$5THIPkmvDF^GtBHK$JgZtt}S?97IGJRRYS6Fo}3Q?0x zV)r?0zVk-QMW@Bv2MyeRt>d=`yYD!ABpfNqMW(Nq8oNJ5r&NMQ=2!26QZ9AoAJ!R=4U0Xgx>bds z6)Wa$^M_{kpEB=8^!;G&=}vQWGly+~K#Cy7N3)qG#{Jr0!rx}#H)*X0Uz$-9LD?8v z2_W%dDbQc@2Jh-sd5VEL7B~120r;FA2r3>vdu?HlZL040L44r)QvtiNp@WMn132zR zDWPVL($LrbEFc82WAf4On@l6{v$yxq1Ims^8F`Nn_L<9y789~qkaM7|IPmChFfUKbRUeo zaXWJh-9dWx$8(JXm@8 z+7RVqo-XhdDk?6@4c6Z6w`xj;Hh>Uj@;juA)x?;BL(da>$cYAl)t2;d^q^(@Wf$WtcgKhZ+Q z5EJT*>{~k=sMAXay@ON7YlzB93GJ#j4r|a{l6_GK5!>ptS}#phj`#_3DWDos@f+xS zb79*@UfPPc=hOMFTmIT{{P}JE7x$kqJZWH8oF!tj`@W<-dEm63W(z!JqW#SYHc&Rf zaBZEO&0Gx*x?M*JHZ|Nzpr8TY+duT<&mzk(r+SYIu^yT6z5KppsiUZX}TKU(T zCIllyO%;&=-lQgdMc1>Z3PU2@k<}*!l`xuaK->&xcj`~Oej@TU)}>4E_O5gry%uPf2;k& z3JHuu+P{qnq`9MAcf7mO(en9%nCsg+t;IgnG<~6Ww^EQ|K`ii^6&x1$XI{fLu7mb! z^zBP#Y+rJ`)@LB^+sr4s#f5XdMvAt77iZQDTU{)Jb zVs_7Iv)Z7}%2%l4JZ)Cr4uM&f`e-*&r+Aqel;m7)T$$A#)sYiLk+&kO#{1W+69QyN zuUHE2E$?dhafjPo=qfXk8qMSe=ZLxZPdX$FOEX-_k2)zzVC@f?cn10`vd*29dWDS_ z=8t{=FI{U-nxbx@fTk18?qHAv>K3LCeaNE;{rPFIg~!$EHmu5)j55S0d00tW7VHiqr-x{uR7??pZ6)||K< z53*rAR*o+?6`Z=tIAzWD0|u>*aGz9I&<%PC@KmGg~&;Dyyx|`%yq94p_d?FA{u(l;%q0?ZC4dw zsX<~tn(q}-vMSll5Urmd2X)`4IB-Ym%%^r2f~&?UJQ-$~I8Y!4+r=Xx!g1XCN7_~E z060PF2@9Axx59k}?5P9TgSUmHn$RZZCbB9kjn6vpr<<(TgPIk zC-2}7rCov8VxQO}ir&2ryeaf5{B7HLs6@1lXGk%uqK(%I=#rMi^;C65+rjMjpehr_ zt)mNQpsUDd)=*9f0BQGxb!iW!ngNJc{%Hp8hrnTB;T7gd?-c;~O`U@ZmS!Wg_kvL6 zkPGCnvg3=S)HF@xTo2(?AoUrEA1(FrIo|^GD~<_dHlsj1p#EElO`|p7dn?*p*4gZV zoCiY2=*toDqZU@RJwHo_s-wU;fj@Id*+<``lOT((rJP_*n}$drZRH?iNQ*MkChmTf z!Q*lcWbS2R`&I?((ik6qtlUqHq266qcK(f4JqtMp0Du;6rJ>SdBpA|SX3q>wb6z%4 zP0bnC;Yu?#cb;JdL7eJk677hOG@+gtc|8BZ_02e_q521&at9m43LomG-8cCxo7aC~ z+)@n4cP>X9Txq6?Cm#g57G5VL3DOqUAPF|P5VhkT?NUEX1rW#Z$)V&0#+OuB=?VFl zGr@sH=3PR{<6-{GA3^=nXwj3xyZF4)2ONjOtAJ{Kluy4%8Qy40D4O_9o-OP8VTqxV4=$j z⩔VxWV1TUtiefufPTFGJcRgu+{#x&gx6bc;SQj1%(VbgVlFntroNUX6UV-{fJ=T z3glJB@_P3Hmnv+(EnPPk)3(()BG))R(`~LgjJn)-Zx_CQTkwH<9To3C0nSp3SyNVD zbnNjO$~@!0I5f>+#{iuiDmK18J5D#ETM{6^{}*WosaQv;^}Q~1+pPXSAs>rxVW6x7 zkPjiHhZlV}A12h$aQ8C}!n*ulp3H<|?gixNvRUoki-IWP4<5`?yo z+w(k#^ngl+2Yc#i;i%F2LG*iT#=xRrVAH}ehTGr{3^r);k{w-zfb;4&pk1YIPTy!Y z073wp*#CtnI38^Skwg$KExYP#6E-(cs=0Sg^#~6>EpmuTpeuYXG}E8;P#c6qORO4r zXiM~81lYvc8r(^r)L+4An%Z+E(7ix(6Dqe?|>>wc+8uAKuW{auLLSm+sj`(ws%=V-Fj- zIn*W4za0AvaN;ivsV<`}^yj}VRN^5h5Kb|iJz3~AYZ0U7`96L2%J^?>pvD_BQg)1O zJWEyMr^h@PYFtKDWBy3StGhfzfg0zJi8DlBPhaSje_5!Be|h@ddfpwe4QWmDg|~Fm z=7wtSFDA_ADC8XWUv%uX)n}Ll8`aC;dO1jUg`Y)&ac?{p;b#CQ3}nLvlk^xD({Jnt zfsGQ=v;Q6c{lr*i;uDTO@H_{iL~f1e1(-1|7_nKc)Za$b<^Bb)dcYx%r}iABNwuAN zz?LVBJ*AVE)6^h~Zfqfa)1T$hM9jxXr_luTkAu!#H|ZKK?G^OqAH4vcp5#LNdz0oK z^%aFe+~KaeaYip-8wxmB7Q8InO#J${(Eb5ao zuKosuHBrGwOPemff>Qu4{^GoOV1pM3k;?RiI{=(=(~GX8@BTv&F#~83`ftiC+*WYG z#TFC-V+-#DoqqetY;9g;DyIBK#5a(DSEJtAVEEftfD!%a1^~_hVnfs%FO73NyZ};J z{8?i(cJBugz*Jc7{+A=ZT9)a7r+m~oReo^K;+iPikZe_C>7k=sYL3J=Zx{n!1q0ra z$zh!B#nb`2V}CMq2po8SGwF|qXj^*WU$zv^c2=n~> z`y=Az2ZI1L+L2j6e-D#?FGy|+dO2TVz|&0-0!bFtQloGGHw&6@CD`dz|Fn7i0}_|b zlDtE8sX6D!Hwni;Apa5wDnw1Xd|-Mm1z_>_y0ir2(ItShm0j8ZuT}#OhYe9nE-niMp9(n;8l_g4(&WGRmJvZS7Sj~~0R%nZCj&b!0db3lx)T-ea68^VJHY?hxT9Z!4g5S2bD`>0=|%a6?nEVcAb3d6@$raEt~_VE9X$1iP| zzzu?<;15Q)LPYgTOYp0!WjFzfSrP( z*!H(Q{!Acsmw=;b>o%qaF#v?ictr`CPY1Z@=*-X@j~!^72xRRi0M_Xmb(z6p)o4pdL2Qr1#nqmSa{&}& zWfh|9$>m+ZUS55OwWS@P;0_+Z2iEwDhv){w2J#vzY`f{A=X$n-HfHm`jTuwj0N%LM zAI#r~z~n?heT}(`oW$AuVL**ob#DvMG5tBU-uBMAR4$EaKNF&6t<&=}#OdDxy$+k? z9+9PgS`wx@S0yh0!VjhQ-2<2WY&(klPm^6fQx6)=@FT%mWd|R_&Hz;kXegkor2n@= zKz;wj<8RSGzDr$*PHwuwqQJJ2`6?OcjHV%e@yq|thpS!u4OM5f?FE>j<}6>}W|ErW z)@2sQS-_i100jSauq*-I1g}&}1uv)s2l)~xD)RJR>&DCL)YyQ&+fN!j@K>N- zdkRwy&!Ri!Rp6XSq3qg58lIB3vME&8C}?P{;>;$Y0JcBFQ-Q8fyIu^x!t5k=LVRRxDr;D#>jiyJgI=}U&#YT z#}WPbXsie_tPhDC-AKE}AJ5(BOA`p_1~0ljoTbL*a-!HT%}iS>x?OE!(qSGdozWII zDNM<^S#~mKP&Ui)2q#C#H}D5orp=4o*IJ^$T_MrnhT+i)sRKD{%02TgkB-34?0iRm z0!)`3Acy|oHA|W@tDFSLp{e}Tj)5E(s1pejP#$Z0f8pGfqwI8bV=Fy+z4YO2qx|q7 zRblV)UFD~4o;6ro7;Fq@miLTPVVK3yB-D|V)YJyF5Aklgk}aprh=O1kw=MnKXK0v6 z-O^Og(0yK?27A)Bpb$|Nr8h_e+LttVW88s#XPN>8tge)TsTZo{J>F-pmb@0rTsPaKDyJ zM|A47WdJRroS?}BjSK$IfA;HClV(S6g9|)g2p=x#-Ng-6IZd^_4-`%Z#mWDv;-#Wg z{L#|Ze&s`4JFpwaEwf2C=V5hlOse*Z>oWqBbh=B26#BT;2L8roLY&IrD&ftibJt~i z_A%q^u&)&VQwF*s!;dXK}FoALb^^_(bSOZ=~^h*YnhpB>%HG?Lu_ z0Q}XL1n~Nze^}28*yi#feaIGg`{Qv&lAUWY0j42W+f~(bsuR2i`|lI}OHdGQCI<^4 zF)lT3)yus96v+mAsmBML(QYJTX4S#~VZ2g=Ge8fme@Kt<5!S>qP^qQ^@ouVk#q^K$ zM4DGOA0Szkq>|`=?0y5JjHT9gxzHnxf6eL7|2=!+7u!?E9vFU!Q}=%i>C5DhFhxA{ z)s(Ta#b*FO1gF`u=zT!`Q$93taXD|7&ksMqCbY7ULVXwSpW{%|Sv(I{E<*br*fOGn z7Sx)*;HvZS|M6>om*SnH!xMN%rV-2bfT({C>6hpiD@O4BzQ|xW%qqtKIBF_Aw)@}3 z1}6K>th#x26hz-EmBnf4(Z6N~>0=pCX@|R-{$fNZc)|5VLk^1#PdrB>q$}yshHcx!K~w-25thXFUp93clQfVHLBW4f8mF_1?q>cuyUVWI z<>9NZ2H&hP;uDtu^Q-DT)%jno<^Rhn2a)m5GRSV2PX8<~lX+9_tGjuX=j$AcqMzj}KN4yGX-L$WK~kt@yKNqQrB7yD z1fdpz=kIQ)^l%=31=qY{XlQsj$_oCluliJ~PQ2&Nuivyr??w_@?b>-xJ5;!oN;P~9 z9}r{Z6kPS$JK3BZt|EGkPyUsvm#a2K+p7aHGW=a(K1M*9pVH#QQ{lBPX@c?1-CFIq zX!2Rfi+gFn_V2wQy3TJ~94z*YuLLN(Y&nQk-N*F4x`b4tfc6VM|H}K@% zKoQ`q4K^8lBh6>axgL)Q+$cKmhb*`bP}SS==eAvRUwfN>0dw5Wni6EsjdIs{>!{Z; zW%@&cOP=*k$f|J%z?q{W7Dn5J>L0VLoL@L_LLI9{x;va2(6xb+F55)({8(mT{C zWd{?l-TlR^>^$iE<3U|}b_bKqAQ;lQLFduxCF98hekT7!6b76XGXr=PT!C%8le2F` z3HjV?9yU9BYm`$^1=6ZYN!i+J9P{fx;5go8=U0C5`J>Iz@ID!G9qzu|QhbS~R*cNu z&`(MbQsnlk)9V$}Pfb&U%QvgdcJ8jc?mF*uE@15~920T=$u52qb1%}NLhEKb+5Q4V zgeYimMvjeW%wa{PZOZG&|{z|mXEU;bCD8m=?Th)clznNaPOYU;B zf>9b%vEQTl3{279nY)QSavlcPj2iKrk4p{=2s(Hgoq|vNrRdwK@WyEjImI|IAaE8S z;z>a)zd}a%xPnDTn#5QHMPx}jyB^xOdr}afsfmVIX2?446 zMATaD!PCZ<4fB3P@jF!hF#Jr~F@oRAsp1~plprfRg-#43UwLs*Yw1C_2p73>K{XYL zCCj)ioc18Z)cJTQd8}+X=WB(|d>)%>7yenauBcOZONe7Th2t(5@s+UVUTuFS77>YD zP{9_aIcQ;V@yz9^uW&y3NR`diw_ZDYD|=$Rp8E07)ASkx-S`KZ;rAw9ueeWv@tu&IC+ z#G6FeGhS@EAbniFY@OWSr9f6gphwVU-fdiE1#VDFdDWOF37wBcJ(nRmDEVe1?UQ;QJ{gjFA;N_OOHP|EtH!KVApmE5Nc%`|)!G;$j! zO@%x+N)W5w^^DtO1KbG{+qMDq`TTICX1T^WUpw`RLC6s`A>#8&)!%Z#19ytV z-?-?kNl)~yo=E-YsZM7&G!h}r5s2%YzF)SP)*os(6JwcIT8OCTwb~LYQQ8{nw;I|i zt5x&#nf`g#s%0Q%d0+)T(N^x!EAX?juc6Q$y#LX*_qwGiz>4t(imH_7n@uBxm&Lmb z&HQ5Js(^Qj<$)^rRn&NNCiHnsx-Z7*ZrbWhm(j@t{Tsdn+Yd-X0f93EsVj;<^UO6f z&9lRfsCgTA=Oe`OtH_T}o0KbPsybATaNBw~@tc%5gi7lXhQC}$bxc0~!6w(@{j!kk zgyp77?OCjvk(ux3M+&2-+g;?Mb4^R?+buQ6e;nP>WUw9K+{z{uDN2&6y6%b_ayd^h zLI@sxeafi$qnaC}QgAnB;oa>YnK3Ep(fYEVg!IBM51vT$R?_u-CnH^lmN#3MvhA8V z*jMHpG}Y&|`Wr|3t=)RL%xQd2N!Cf2TXa^(mWT7DeCP*j-yZGkH|h0zdMLAj#j_7Q zt#K?I6VoNNx<%Fx6xZWcQZB6+&NOTny-N(6Unb%(KlqX8(iJ0>xiejPImKzJUZiUJs~st*dGNA>erz-d1FQK zsyPPPc}J8UwYl%A?){k9&}1UJ^@`wop+QkByL5{8Q6E>b8ELQ|gTgE!iSzsNiGuU@CzXF=g@k-p z!$K%a$_QRUDEjROB(rUA(nrsk^Xeu!4u?0#qZ^i!-zlEDiH$$PqOlpkTc20t^E*>> z$ArVqAFQ$XDg;*h5=D7;vFfd~oWDT#A=N4H={W`8&Hw_gV%R5eiR+7}hP$e)_rmxu zdxuDe$+h~AO4lJDag!jLzTf>JVOOY%y03-O>mr*US56%d($SJKS|J60~?%Tn3^(p^1dT?8NcX_+si;o24Ge2%hp*Q9Nms()N85cwKanFhbUh-(f z=&GqnLYe;4!8p}CWN|wx2YWe%9OvxnHtn_q8yyvJ8$2C*?!lR%pbm<)g}+67cVAOu z%X)9i;i4FXLeJ4AANM~P_&+$57u6>VH-T&w!7(2mqh=3F2>N2@WspO0ez`q6ssI3% ze0(s+llX&YbanT+W0rN^={v_cCJv{_oy&kqsZpNlXB%D!JEC~HN>ZiA*xZzb54y72 zSWCpu_FVb?;&t=bNeIza_+d|z9%~|zHMuo`RH3riHhA)xT4E((S=lZvYLrNjxq(#U zm%TFNv3?Dk6^-!LW{ESDW!ehQ@84p@~Zgjk`&g5^15=>$@O7jS@2$C-xd5^OcJSs06g+x++>1}!Jc{^8MnqI z!uQhFvE~wsrCizmOysU9pZ4cNfE3+g79_fEOg7KRWsec~^UgB8l$vTc1-fg#AW}Oi zW>B5a>y2IerJ!bl zcJ$crXWXn#F`rwpV}LN$Z%U(i8W=exz@d<>`R| zBg{<09S3lg6PtBwG+-nod6V@PQHDI(yox|YXm+`mS)30|)KKEiZ+xhJ?(uh z(S_Hu{Xj!p?&JC$5~!??D_&2rZ$_W)$X*TRp@^3Z`C2CBIOT90($6~(ODfhL+(^pj zoceyG6)!X^57aau|M1Bw&oybaA%!_RlaF`GmWG`2S=J2U*s5jYLGK-J7U4VkK{mUD6HdqOwb0 zk{2HiPaxp=GffN%cbMRR+z>(~S=SAnvQ*<#kq!BnOt1t!gVgO+1H3aMQ$r!RU_ye; zacK)&<*1tFac|N{2{v6jwmY4dwnN`JRRbz4sHBqDuA6YuKxQoJ+yjG`KHDvQ0P%_5 z!MeteYj!1Yh7nmsW-lrwnXrZ%<>`Mf)%wzVgXBR9jo#gV(jY6VZ&5k3cs2%Cw4=?X zA1|}eC|%pM<}%y8-aM0U^C(f-?aezARN1?0>ACs__zvv_j_nic7mPlY{zt?Lz1Gv_rL$BfjfRPEMF#pcx7=WNXoxx_I2m z7DxHvKIV$HRQad70_A0{qrmYb(%6X!-^1U{FR!I<5|>3>LW5{RbEI+d*~u1oVR#rBbl!yZ-l>Yi6q*ogY8Xe!BN; z?Ld>=Z2J3|kGHC(v=04B=pJ&%39+GOCET>@_Qr?cJV_<&ELykGGtD!$yH564=H^$Pt?Bg{`LOBKGf0sciVY)MlB{{D zAXk35_C|cSVjS*cytK7g&(W*Gj9mUd!wuC543>`E8#ZfFDaG5?XwgX0jUg(7y$mSr z89bv<_hftgXTT0A&_i6rB0fKP9)XmtT6-M75QX3sG2mNwZQ{74;bMBCXFJ<1i^?&n zGW?UO__AM#5awvw4&v#Eqm0be7V~BP4EvwM}BZF=l z{5ZU&MYBn3HXw7>#bvc@bKN|UM_lN#%C#51e6T~KmQn^K!ZX&p#O8A{5;PVG**Lz< zDUPcFl9*okQTPIfemH6Us4z@z0GD7gIgk$;2jWNX&vs_o%=s>fs)nTVQO)E)AA4Zp zS&vE;Qq!nnR*KZ{>UH>z40m18G9!GrV(?2e3_t&Z=s#!!y!F zm`xugd1-X*b{ScoyruO`M@`+$no)=bjK}X3)w8V7y)DBaHvZ3;KY(leb?VKI8kD|H zm+uzd>Z)>b$xo#PU2#5c^};i@DV|H5fjCsd-Jit|yu|ej3f^t{JVS<*-~L*=ykXe~ z=j&2m^$gLK`clgpAChdYcJG;4Z$)ARt^j2|8f&JJ|LnW5x5(&ld6;V*kyIy&H^yv6 zT~j^UNW}TxG1FTVqMYrRWq*`+Lw~*ySqO=Tv>9V!a^qp{%9e}PzGVntCIh&Sl09}I zeGM+p1|Iy8GXDcV{y3?OK?Ki7#oB*yw3d8p_p-yU%J#!;pBK%O0ewC^kFQcx z;#B#=P%YL8WdHVwwV^6PMvU=OKB4@>-p}v%mG=!;Ric-F_Sfc2lZ&3j2GlC~Q+Ce~ zRs>40rrzk3vT+hpanC^R^q^?AzMEf76s?6qOj?~*ssr3M3s0Ms7I>%4q}%xR zY@a1O4lOg@0}B_FYk6CxydKKtiS=~huYU!vyk#{C)sE;(_jf2ggA{E0(oV=^{_^HA z%dRW&<@m94hXT#eW-P*eczH8hvri9y>QGF4$fU#(JML9p#TVT?i9cVYW%g|tW$-Rg z_KRVwj_?wE3i#P5xFGT=iFYM3Yn9#)p?&2p+HD->N)hp}jbMVS5qzh@lu>LoQcqWR ze2iP-QjY=DR=qWE6j`=|Avly}t#G(11VdKpI&q2* zD}YE%Uai&H=Jo04#mpanX1qZrNssD>#2P@rbx@jX{p~u0&wp_e(bfl=y&OPJi6yO} zGh&Y@yI%bfC2p^xt}kx`yk{AwPU13#qGt1{1hK)&exfpOAOi^n(@l%FbhiL3@d6$!^KF7!2f;pdXA%^T2u zi*1;o#wL@xfcL_CKySqgs5>S3r>?4nfmA@yYIJj~{H7p2i}fQOwkdZ3xf^ulwF&Fl zF*BLqoG>M28gWdVeC_N|5a=72*xgirw{W-)K-(%|x>kOMAqR?fzZx}0%c5WNnm9CRg5zOPFKZvoYK^Ip<; z>Y@(Xt8CWZx0s&|M=`zyinXM^NIsf=0M5M~<_hh^=}XWsR}iL6y&X`iTj|a|x-)D} zq7UUvfgbC1yO_TBARBgmU5#2JXF=v8-w1_NzT?+6Jl-(V06r-s@q3e z`e`lqJ>xn87n>+oE^H&uyyZ1Nvb5{OLURcs0yv>p9Y{UqKR;Toy&Wou4Z0E$6_w<_ zC{}V$lq8APmyyZRzIpTA4>3eo7BM_~C%@1mnK0`^Y&a{G&IaSk4#$TTqa^ZZq*R7< zols|Qa+lMB;$)*sk9D}^&D4&5CwSf~<*N9ks9?g3Arm`ls< zKDQI;u#R%MX%nxhfVGTQ>pfi5G0UX{|KpV;{~!tlQXoOL^p>Rtv))pto2Ad2+_i%h zA0ePz%zCgb{KSLKg6Bu{0zUGe%ZUc}jSf#YeZj-jXQkBg1&KFvHNs+2cCY}&YdhIG znc!hy*PTOZm8@fZ>OA?LHMF_!d+yHA!M%a`hPmaH8(Xm&__Qq*Ew#-@DV2)@v$d0n zX;-7f5q*4H)09i-!5BsrTgz3!Qa>Nbft85e(3vt7*zzK$~$J*cW!#ivoHgzeiU#i z$yJTGdP+CjP~f$5%QkNelXIoZ@?c|xKb94=1Wt=Zad{R10P{-sU(7X)Kej$d6dIl8 zvX;X)ivWi~j*tZNXHp~!%%1%ssXO(+AMF;sx71A1aQm1i``I0s1Rqpqy;R6N86JFQ z4-*M`yv?nyzIq%AI@$P&;5%)htfwVF)^~d5`Hly0dnrQ?=BBrQQ>(mP)D)84_l6^k zAomgWi>E*vFf~95Qm((R0gb@$9RhXUgj|BzWh&J+GV4REHjVmQ2d;D8Kga(*pwew& zKs;63CVY^{D$n{ijXK+Q_;X(x%yOxnkG16FSCrXLidNKwS{Z`X!t=@MylgenWl?&O zqPp}9BqgjcpA@kK*4I)N%h4hH^0k-Gz^-l6yP>KMlalp~Jg)va;1b-$3DWDdRTkTa#wF?r40k!@+vCUf_q=hS6W^?7CQOn~e5AXGKjplAP z<-PP)_8YN8tMwR0TlKL)Yus{MKwtR1u0({-JPQbC>bLtZ;%Cmi6iamfQZL*M8ho{V zv_#8Wb0o?m!**kEuE~P$$;Nowq+1iQ$Z~Tf7p4As*QMXk@!Wm+4IeIXgtU^w6GBC} zhOp+X^`BInWA4gsb@Y=KSQ^_vsvV@5LV16=2vT!MZbL3b5%H8Kc>{3iHtSzHQi